From 1745c1274b274a994a92312054d8951ce1c499d0 Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 23 Aug 2016 18:12:59 -0400 Subject: [PATCH] NIFI-2608 This closes #930. restructured the ConsumeKafka processor to follow new consumer API. Made nar classloading more precise to support spawned threads NIFI-2660. --- nifi-assembly/pom.xml | 9 +- .../init/ControllerServiceInitializer.java | 4 +- .../init/ProcessorInitializer.java | 4 +- .../init/ReportingTaskingInitializer.java | 4 +- .../AbstractConfiguredComponent.java | 18 +- .../nifi/controller/FlowController.java | 28 +- .../controller/StandardProcessorNode.java | 22 +- .../EventDrivenSchedulingAgent.java | 8 +- .../scheduling/StandardProcessScheduler.java | 6 +- .../StandardControllerServiceProvider.java | 6 +- .../tasks/ContinuallyRunConnectableTask.java | 4 +- .../tasks/ContinuallyRunProcessorTask.java | 2 +- .../tasks/ReportingTaskWrapper.java | 4 +- .../nifi/groups/StandardProcessGroup.java | 6 +- .../org/apache/nifi/nar/NarCloseable.java | 22 +- .../nifi/web/controller/ControllerFacade.java | 2 +- .../pom.xml | 4 +- .../src/main/resources/META-INF/LICENSE | 0 .../src/main/resources/META-INF/NOTICE | 0 .../nifi-kafka-0-10-processors/pom.xml | 84 +++ .../kafka/pubsub/ConsumeKafka_0_10.java | 407 ++++++++++++++ .../kafka/pubsub/ConsumerLease.java | 68 +++ .../processors/kafka/pubsub/ConsumerPool.java | 266 +++++++++ .../kafka/pubsub/KafkaProcessorUtils.java | 264 +++++++++ .../kafka/pubsub/KafkaPublisher.java | 27 +- .../processors/kafka/pubsub/Partitioners.java | 1 + .../kafka/pubsub/PublishKafka_0_10.java | 516 ++++++++++++++++++ .../kafka/pubsub/PublishingContext.java | 0 .../org.apache.nifi.processor.Processor | 16 + .../additionalDetails.html | 33 ++ .../additionalDetails.html | 47 ++ .../kafka/pubsub/ConsumeKafkaTest.java | 496 +++++++++++++++++ .../kafka/pubsub/ConsumerPoolTest.java | 124 +++++ .../kafka/pubsub/KafkaPublisherTest.java | 8 +- .../kafka/pubsub/PublishKafkaTest.java | 329 +++++++++++ .../kafka/pubsub/PublishingContextTest.java | 8 +- .../kafka/pubsub/StubPublishKafka.java | 133 +++++ .../processors/kafka/pubsub/TestUtils.java | 1 - .../processors/kafka/test/EmbeddedKafka.java | 0 .../test/EmbeddedKafkaProducerHelper.java | 0 .../src/test/resources/log4j.properties | 0 .../src/test/resources/server.properties | 0 .../src/test/resources/zookeeper.properties | 0 .../pom.xml | 4 +- .../src/main/resources/META-INF/LICENSE | 0 .../src/main/resources/META-INF/NOTICE | 0 .../pom.xml | 10 +- .../kafka/AbstractKafkaProcessor.java | 0 .../nifi/processors/kafka/GetKafka.java | 0 .../nifi/processors/kafka/KafkaPublisher.java | 0 .../nifi/processors/kafka/KafkaUtils.java | 0 .../nifi/processors/kafka/Partitioners.java | 0 .../processors/kafka/PublishingContext.java | 0 .../nifi/processors/kafka/PutKafka.java | 0 .../org.apache.nifi.processor.Processor | 0 .../additionalDetails.html | 0 .../additionalDetails.html | 0 .../kafka/GetKafkaIntegrationTests.java | 0 .../processors/kafka/KafkaPublisherTest.java | 0 .../nifi/processors/kafka/PutKafkaTest.java | 0 .../nifi/processors/kafka/TestGetKafka.java | 0 .../processors/kafka/test/EmbeddedKafka.java | 0 .../test/EmbeddedKafkaProducerHelper.java | 0 .../src/test/resources/log4j.properties | 0 .../src/test/resources/server.properties | 0 .../src/test/resources/zookeeper.properties | 0 .../nifi-kafka-0-9-nar/pom.xml | 40 ++ .../src/main/resources/META-INF/LICENSE | 299 ++++++++++ .../src/main/resources/META-INF/NOTICE | 72 +++ .../pom.xml | 11 +- .../processors/kafka/pubsub/ConsumeKafka.java | 408 ++++++++++++++ .../kafka/pubsub/ConsumerLease.java | 68 +++ .../processors/kafka/pubsub/ConsumerPool.java | 266 +++++++++ .../kafka/pubsub/KafkaProcessorUtils.java | 266 +++++++++ .../kafka/pubsub/KafkaPublisher.java | 236 ++++++++ .../processors/kafka/pubsub/Partitioners.java | 62 +++ .../processors/kafka/pubsub/PublishKafka.java | 274 ++++++++-- .../kafka/pubsub/PublishingContext.java | 139 +++++ .../org.apache.nifi.processor.Processor | 0 .../additionalDetails.html | 0 .../additionalDetails.html | 0 .../kafka/pubsub/ConsumeKafkaTest.java | 496 +++++++++++++++++ .../kafka/pubsub/ConsumerPoolTest.java | 124 +++++ .../kafka/pubsub/KafkaPublisherTest.java | 306 +++++++++++ .../kafka/pubsub/PublishKafkaTest.java | 65 +-- .../kafka/pubsub/PublishingContextTest.java | 106 ++++ .../kafka/pubsub/StubPublishKafka.java | 11 +- .../processors/kafka/pubsub/TestUtils.java | 45 ++ .../processors/kafka/test/EmbeddedKafka.java | 226 ++++++++ .../test/EmbeddedKafkaProducerHelper.java | 110 ++++ .../src/test/resources/log4j.properties | 21 + .../src/test/resources/server.properties | 121 ++++ .../src/test/resources/zookeeper.properties | 20 + .../kafka/pubsub/AbstractKafkaProcessor.java | 400 -------------- .../processors/kafka/pubsub/ConsumeKafka.java | 296 ---------- .../processors/kafka/pubsub/ConsumerPool.java | 148 ----- .../kafka/pubsub/ConsumerResource.java | 76 --- .../AbstractKafkaProcessorLifecycleTest.java | 456 ---------------- .../kafka/pubsub/ConsumeKafkaTest.java | 188 ------- .../kafka/pubsub/StubConsumeKafka.java | 71 --- nifi-nar-bundles/nifi-kafka-bundle/pom.xml | 19 +- pom.xml | 10 +- 102 files changed, 6605 insertions(+), 1846 deletions(-) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-nar => nifi-kafka-0-10-nar}/pom.xml (93%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-nar => nifi-kafka-0-10-nar}/src/main/resources/META-INF/LICENSE (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-nar => nifi-kafka-0-10-nar}/src/main/resources/META-INF/NOTICE (100%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java (92%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java (99%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java (100%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java (96%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java (93%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java (99%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-10-processors}/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-10-processors}/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/resources/log4j.properties (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/resources/server.properties (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-10-processors}/src/test/resources/zookeeper.properties (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-nar => nifi-kafka-0-8-nar}/pom.xml (93%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-nar => nifi-kafka-0-8-nar}/src/main/resources/META-INF/LICENSE (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-nar => nifi-kafka-0-8-nar}/src/main/resources/META-INF/NOTICE (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/pom.xml (93%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/Partitioners.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/PublishingContext.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/GetKafkaIntegrationTests.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/PutKafkaTest.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-8-processors}/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/resources/log4j.properties (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/resources/server.properties (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-processors => nifi-kafka-0-8-processors}/src/test/resources/zookeeper.properties (100%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/LICENSE create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/pom.xml (93%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java (57%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html (100%) rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html (100%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java (86%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java rename nifi-nar-bundles/nifi-kafka-bundle/{nifi-kafka-pubsub-processors => nifi-kafka-0-9-processors}/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java (94%) create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/log4j.properties create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/server.properties create mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/zookeeper.properties delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerResource.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessorLifecycleTest.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java delete mode 100644 nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubConsumeKafka.java diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 44a03b5002..334b7b526e 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -163,12 +163,17 @@ language governing permissions and limitations under the License. --> org.apache.nifi - nifi-kafka-nar + nifi-kafka-0-8-nar nar org.apache.nifi - nifi-kafka-pubsub-nar + nifi-kafka-0-9-nar + nar + + + org.apache.nifi + nifi-kafka-0-10-nar nar diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java index a6c397a8a5..c641afe7c0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java @@ -39,14 +39,14 @@ public class ControllerServiceInitializer implements ConfigurableComponentInitia public void initialize(ConfigurableComponent component) throws InitializationException { ControllerService controllerService = (ControllerService) component; - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { controllerService.initialize(new MockControllerServiceInitializationContext()); } } @Override public void teardown(ConfigurableComponent component) { - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { ControllerService controllerService = (ControllerService) component; final ComponentLog logger = new MockComponentLogger(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java index 745c27c53c..221f9e50eb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java @@ -37,7 +37,7 @@ public class ProcessorInitializer implements ConfigurableComponentInitializer { @Override public void initialize(ConfigurableComponent component) { Processor processor = (Processor) component; - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { processor.initialize(new MockProcessorInitializationContext()); } } @@ -45,7 +45,7 @@ public class ProcessorInitializer implements ConfigurableComponentInitializer { @Override public void teardown(ConfigurableComponent component) { Processor processor = (Processor) component; - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { final ComponentLog logger = new MockComponentLogger(); final MockProcessContext context = new MockProcessContext(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java index 3cf86c9b19..8233e2ea8b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java @@ -37,7 +37,7 @@ public class ReportingTaskingInitializer implements ConfigurableComponentInitial @Override public void initialize(ConfigurableComponent component) throws InitializationException { ReportingTask reportingTask = (ReportingTask) component; - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { reportingTask.initialize(new MockReportingInitializationContext()); } } @@ -45,7 +45,7 @@ public class ReportingTaskingInitializer implements ConfigurableComponentInitial @Override public void teardown(ConfigurableComponent component) { ReportingTask reportingTask = (ReportingTask) component; - try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { final MockConfigurationContext context = new MockConfigurationContext(); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java index 0454a2f1b1..646005030e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java @@ -99,7 +99,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone try { verifyModifiable(); - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); final String oldValue = properties.put(descriptor, value); @@ -151,7 +151,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone try { verifyModifiable(); - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); String value = null; if (!descriptor.isRequired() && (value = properties.remove(descriptor)) != null) { @@ -182,7 +182,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone @Override public Map getProperties() { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { final List supported = component.getPropertyDescriptors(); if (supported == null || supported.isEmpty()) { return Collections.unmodifiableMap(properties); @@ -226,35 +226,35 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone @Override public String toString() { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { return component.toString(); } } @Override public Collection validate(final ValidationContext context) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { return component.validate(context); } } @Override public PropertyDescriptor getPropertyDescriptor(final String name) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { return component.getPropertyDescriptor(name); } } @Override public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { component.onPropertyModified(descriptor, oldValue, newValue); } } @Override public List getPropertyDescriptors() { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { return component.getPropertyDescriptors(); } } @@ -286,7 +286,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone serviceIdentifiersNotToValidate, getProperties(), getAnnotationData(), getProcessGroupIdentifier(), getIdentifier()); final Collection validationResults; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass())) { validationResults = component.validate(validationContext); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 3acba9487c..fee44ca15d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -733,7 +733,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R private void notifyComponentsConfigurationRestored() { for (final ProcessorNode procNode : getGroup(getRootGroupId()).findAllProcessors()) { final Processor processor = procNode.getProcessor(); - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); } } @@ -741,7 +741,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R for (final ControllerServiceNode serviceNode : getAllControllerServices()) { final ControllerService service = serviceNode.getControllerServiceImplementation(); - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(service.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); } } @@ -749,7 +749,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R for (final ReportingTaskNode taskNode : getAllReportingTasks()) { final ReportingTask task = taskNode.getReportingTask(); - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(task.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, task); } } @@ -1062,7 +1062,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ProcessorLogObserver(getBulletinRepository(), procNode)); if (firstTimeAdded) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); } catch (final Exception e) { logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID); @@ -1070,7 +1070,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R } if (firstTimeAdded) { - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, procNode.getProcessor()); } } @@ -1330,7 +1330,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R // invoke any methods annotated with @OnShutdown on Controller Services for (final ControllerServiceNode serviceNode : getAllControllerServices()) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(serviceNode.getControllerServiceImplementation().getClass())) { final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null, variableRegistry); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext); } @@ -1339,7 +1339,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R // invoke any methods annotated with @OnShutdown on Reporting Tasks for (final ReportingTaskNode taskNode : getAllReportingTasks()) { final ConfigurationContext configContext = taskNode.getConfigurationContext(); - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), configContext); } } @@ -2877,7 +2877,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + type, ie); } - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, taskNode.getReportingTask()); } catch (final Exception e) { @@ -2931,7 +2931,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R reportingTaskNode.verifyCanDelete(); - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTaskNode.getReportingTask().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); } @@ -2968,7 +2968,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R if (firstTimeAdded) { final ControllerService service = serviceNode.getControllerServiceImplementation(); - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(service.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); } } @@ -3087,7 +3087,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R service.verifyCanDelete(); - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(service.getControllerServiceImplementation().getClass())) { final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext); } @@ -3450,17 +3450,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final PrimaryNodeState nodeState = primary ? PrimaryNodeState.ELECTED_PRIMARY_NODE : PrimaryNodeState.PRIMARY_NODE_REVOKED; final ProcessGroup rootGroup = getGroup(getRootGroupId()); for (final ProcessorNode procNode : rootGroup.findAllProcessors()) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, procNode.getProcessor(), nodeState); } } for (final ControllerServiceNode serviceNode : getAllControllerServices()) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(serviceNode.getControllerServiceImplementation().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, serviceNode.getControllerServiceImplementation(), nodeState); } } for (final ReportingTaskNode reportingTaskNode : getAllReportingTasks()) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(reportingTaskNode.getReportingTask().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, reportingTaskNode.getReportingTask(), nodeState); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index cb12ab0f5f..42790fdca9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -811,7 +811,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable Relationship returnRel = specRel; final Set relationships; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass())) { relationships = processor.getRelationships(); } @@ -857,7 +857,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable public Set getUndefinedRelationships() { final Set undefined = new HashSet<>(); final Set relationships; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass())) { relationships = processor.getRelationships(); } @@ -913,7 +913,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable .newValidationContext(getProperties(), getAnnotationData(), getProcessGroupIdentifier(), getIdentifier()); final Collection validationResults; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getProcessor().getClass())) { validationResults = getProcessor().validate(validationContext); } @@ -960,7 +960,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable .newValidationContext(getProperties(), getAnnotationData(), getProcessGroup().getIdentifier(), getIdentifier()); final Collection validationResults; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getProcessor().getClass())) { validationResults = getProcessor().validate(validationContext); } @@ -1036,14 +1036,14 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable @Override public Collection getRelationships() { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getProcessor().getClass())) { return getProcessor().getRelationships(); } } @Override public String toString() { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getProcessor().getClass())) { return getProcessor().toString(); } } @@ -1060,7 +1060,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable @Override public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) { - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass())) { processor.onTrigger(context, sessionFactory); } } @@ -1240,7 +1240,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable invokeTaskAsCancelableFuture(schedulingAgentCallback, new Callable() { @Override public Void call() throws Exception { - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext); return null; } @@ -1250,7 +1250,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable if (scheduledState.compareAndSet(ScheduledState.STARTING, ScheduledState.RUNNING)) { schedulingAgentCallback.trigger(); // callback provided by StandardProcessScheduler to essentially initiate component's onTrigger() cycle } else { // can only happen if stopProcessor was called before service was transitioned to RUNNING state - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext); } scheduledState.set(ScheduledState.STOPPED); @@ -1325,7 +1325,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable try { if (scheduleState.isScheduled()) { schedulingAgent.unschedule(StandardProcessorNode.this, scheduleState); - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext); } } @@ -1334,7 +1334,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable // performing the lifecycle actions counts as 1 thread. final boolean allThreadsComplete = scheduleState.getActiveThreadCount() == 1; if (allThreadsComplete) { - try (final NarCloseable nc = NarCloseable.withNarLoader()) { + try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index 091d1f6335..860b4da070 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -287,7 +287,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { } try { - try (final AutoCloseable ncl = NarCloseable.withNarLoader()) { + try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(worker.getClass())) { worker.onTrigger(processContext, sessionFactory); } catch (final ProcessException pe) { logger.error("{} failed to process session due to {}", worker, pe.toString()); @@ -305,7 +305,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { } } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(worker.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker, processContext); } } @@ -328,7 +328,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { } try { - try (final AutoCloseable ncl = NarCloseable.withNarLoader()) { + try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(worker.getProcessor().getClass())) { worker.onTrigger(processContext, sessionFactory); } catch (final ProcessException pe) { final ComponentLog procLog = new SimpleProcessLogger(worker.getIdentifier(), worker.getProcessor()); @@ -347,7 +347,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { // if the processor is no longer scheduled to run and this is the last thread, // invoke the OnStopped methods if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(worker.getProcessor().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker.getProcessor(), processContext); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 0a76e4f272..305fad0552 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -209,7 +209,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { return; } - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTask.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, reportingTask, taskNode.getConfigurationContext()); } @@ -262,7 +262,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { scheduleState.setScheduled(false); try { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTask.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, configurationContext); } } catch (final Exception e) { @@ -436,7 +436,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { if (!state.isScheduled() && state.getActiveThreadCount() == 0 && state.mustCallOnStoppedMethods()) { final ConnectableProcessContext processContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier())); - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(connectable.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index 9fd1ca2466..b4d7e26826 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -165,7 +165,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi final boolean disabled = state != ControllerServiceState.ENABLED; // only allow method call if service state is ENABLED. if (disabled && !validDisabledMethods.contains(method)) { // Use nar class loader here because we are implicitly calling toString() on the original implementation. - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(originalService.getClass())) { throw new IllegalStateException("Cannot invoke method " + method + " on Controller Service " + originalService.getIdentifier() + " because the Controller Service is disabled"); } catch (final Throwable e) { @@ -173,7 +173,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } } - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(originalService.getClass())) { return method.invoke(originalService, args); } catch (final InvocationTargetException e) { // If the ControllerService throws an Exception, it'll be wrapped in an InvocationTargetException. We want @@ -201,7 +201,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi serviceNode.setName(rawClass.getSimpleName()); if (firstTimeAdded) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(originalService.getClass())) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, originalService); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + originalService, e); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index 8f1623ed31..b5d6a4df61 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -76,7 +76,7 @@ public class ContinuallyRunConnectableTask implements Callable { if (shouldRun) { scheduleState.incrementActiveThreadCount(); try { - try (final AutoCloseable ncl = NarCloseable.withNarLoader()) { + try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(connectable.getClass())) { connectable.onTrigger(processContext, sessionFactory); } catch (final ProcessException pe) { logger.error("{} failed to process session due to {}", connectable, pe.toString()); @@ -93,7 +93,7 @@ public class ContinuallyRunConnectableTask implements Callable { } } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(connectable.getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java index 6d65604751..f3e8474838 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java @@ -130,7 +130,7 @@ public class ContinuallyRunProcessorTask implements Callable { final long finishNanos = startNanos + batchNanos; int invocationCount = 0; try { - try (final AutoCloseable ncl = NarCloseable.withNarLoader()) { + try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass())) { boolean shouldRun = true; while (shouldRun) { procNode.onTrigger(processContext, sessionFactory); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index eb087a4331..ea93db15ba 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -37,7 +37,7 @@ public class ReportingTaskWrapper implements Runnable { @Override public synchronized void run() { scheduleState.incrementActiveThreadCount(); - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass())) { taskNode.getReportingTask().onTrigger(taskNode.getReportingContext()); } catch (final Throwable t) { final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), taskNode.getReportingTask()); @@ -50,7 +50,7 @@ public class ReportingTaskWrapper implements Runnable { // if the reporting task is no longer scheduled to run and this is the last thread, // invoke the OnStopped methods if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index f1c12c121c..901c42ee8e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -348,7 +348,7 @@ public final class StandardProcessGroup implements ProcessGroup { private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(node.getProcessor().getClass())) { final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()), variableRegistry); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext); } @@ -707,7 +707,7 @@ public final class StandardProcessGroup implements ProcessGroup { conn.verifyCanDelete(); } - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getProcessor().getClass())) { final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()), variableRegistry); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { @@ -1846,7 +1846,7 @@ public final class StandardProcessGroup implements ProcessGroup { service.verifyCanDelete(); - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(service.getControllerServiceImplementation().getClass())) { final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java index c65501cc9d..116b06915c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; * */ public class NarCloseable implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(NarCloseable.class); public static NarCloseable withNarLoader() { @@ -34,8 +35,25 @@ public class NarCloseable implements Closeable { } /** - * Creates a Closeable object that can be used to to switch to current class loader to the framework class loader - * and will automatically set the ClassLoader back to the previous class loader when closed + * Sets the current thread context class loader to the specific appropriate + * Nar class loader for the given configurable component. Restores to the + * previous classloader once complete. If the given class is not assignable + * from ConfigurableComponent then the NarThreadContextClassLoader is used. + * + * @param componentClass componentClass + * @return NarCloseable with current thread context classloader jailed to + * the nar of the component + */ + public static NarCloseable withComponentNarLoader(final Class componentClass) { + final ClassLoader current = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(componentClass.getClassLoader()); + return new NarCloseable(current); + } + + /** + * Creates a Closeable object that can be used to to switch to current class + * loader to the framework class loader and will automatically set the + * ClassLoader back to the previous class loader when closed * * @return a NarCloseable */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index fe0bff19f3..4ded68129a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -1641,7 +1641,7 @@ public class ControllerFacade implements Authorizable { final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController, variableRegistry); // search the processor using the appropriate thread context classloader - try (final NarCloseable x = NarCloseable.withNarLoader()) { + try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getClass())) { final Collection searchResults = searchable.search(context); if (CollectionUtils.isNotEmpty(searchResults)) { for (final SearchResult searchResult : searchResults) { diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml similarity index 93% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml index cb3be3854b..a35dcddede 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/pom.xml @@ -19,7 +19,7 @@ nifi-kafka-bundle 1.0.0-SNAPSHOT - nifi-kafka-pubsub-nar + nifi-kafka-0-10-nar nar NiFi NAR for interacting with Apache Kafka @@ -29,7 +29,7 @@ org.apache.nifi - nifi-kafka-pubsub-processors + nifi-kafka-0-10-processors org.apache.nifi diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/LICENSE similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/src/main/resources/META-INF/LICENSE rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/LICENSE diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/src/main/resources/META-INF/NOTICE rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-nar/src/main/resources/META-INF/NOTICE diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml new file mode 100644 index 0000000000..7b4c3eb77b --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/pom.xml @@ -0,0 +1,84 @@ + + + + org.apache.nifi + nifi-kafka-bundle + 1.0.0-SNAPSHOT + + 4.0.0 + nifi-kafka-0-10-processors + jar + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-utils + + + org.apache.nifi + nifi-ssl-context-service-api + + + org.apache.kafka + kafka-clients + 0.10.0.1 + + + org.apache.kafka + kafka_2.10 + 0.10.0.1 + test + + + + javax.jms + jms + + + com.sun.jdmk + jmxtools + + + com.sun.jmx + jmxri + + + + + org.apache.nifi + nifi-mock + test + + + commons-io + commons-io + test + + + org.slf4j + slf4j-simple + test + + + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java new file mode 100644 index 0000000000..53d6fcd43c --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.xml.bind.DatatypeConverter; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURITY_PROTOCOL; + +@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.10 Consumer API. " + + " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring" + + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can. In the mean time" + + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.") +@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.10"}) +@WritesAttributes({ + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY_HEX, description = "The hex encoded key of message if present and if single message"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic the message or message bundle is from") +}) +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.", + description = "These properties will be added on the Kafka configuration after loading any provided configuration properties." + + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged." + + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ") +public class ConsumeKafka_0_10 extends AbstractProcessor { + + private static final long FIVE_MB = 5L * 1024L * 1024L; + + static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset"); + + static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset"); + + static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group"); + + static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name(s)") + .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma seperated.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder() + .name(ConsumerConfig.GROUP_ID_CONFIG) + .displayName("Group ID") + .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder() + .name(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + .displayName("Offset Reset") + .description("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any " + + "more on the server (e.g. because that data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + .required(true) + .allowableValues(OFFSET_EARLIEST, OFFSET_LATEST, OFFSET_NONE) + .defaultValue(OFFSET_LATEST.getValue()) + .build(); + + static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder() + .name("message-demarcator") + .displayName("Message Demarcator") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains " + + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use " + + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received " + + "will result in a single FlowFile which " + + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS") + .build(); + static final PropertyDescriptor MAX_POLL_RECORDS = new PropertyDescriptor.Builder() + .name("max.poll.records") + .displayName("Max Poll Records") + .description("Specifies the maximum number of records Kafka should return in a single poll.") + .required(false) + .defaultValue("10000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .build(); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles received from Kafka. Depending on demarcation strategy it is a flow file per message or a bundle of messages grouped by topic and partition.") + .build(); + + static final List DESCRIPTORS; + static final Set RELATIONSHIPS; + + private volatile byte[] demarcatorBytes = null; + private volatile ConsumerPool consumerPool = null; + + static { + List descriptors = new ArrayList<>(); + descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + descriptors.add(TOPICS); + descriptors.add(GROUP_ID); + descriptors.add(AUTO_OFFSET_RESET); + descriptors.add(MESSAGE_DEMARCATOR); + descriptors.add(MAX_POLL_RECORDS); + DESCRIPTORS = Collections.unmodifiableList(descriptors); + RELATIONSHIPS = Collections.singleton(REL_SUCCESS); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @OnScheduled + public void prepareProcessing(final ProcessContext context) { + this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() + ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) + : null; + } + + @OnStopped + public void close() { + demarcatorBytes = null; + final ConsumerPool pool = consumerPool; + consumerPool = null; + if (pool != null) { + pool.close(); + } + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.") + .name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class)).dynamic(true) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + return KafkaProcessorUtils.validateCommonProperties(validationContext); + } + + private synchronized ConsumerPool getConsumerPool(final ProcessContext context) { + ConsumerPool pool = consumerPool; + if (pool != null) { + return pool; + } + + final Map props = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props); + final String topicListing = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue(); + final List topics = new ArrayList<>(); + for (final String topic : topicListing.split(",", 100)) { + final String trimmedName = topic.trim(); + if (!trimmedName.isEmpty()) { + topics.add(trimmedName); + } + } + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + return consumerPool = createConsumerPool(context.getMaxConcurrentTasks(), topics, props, getLogger()); + } + + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return new ConsumerPool(maxLeases, topics, props, log); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final long startTimeNanos = System.nanoTime(); + final ConsumerPool pool = getConsumerPool(context); + if (pool == null) { + context.yield(); + return; + } + final Map>> partitionRecordMap = new HashMap<>(); + + try (final ConsumerLease lease = pool.obtainConsumer()) { + try { + if (lease == null) { + context.yield(); + return; + } + + final boolean foundData = gatherDataFromKafka(lease, partitionRecordMap, context); + if (!foundData) { + session.rollback(); + return; + } + + writeSessionData(context, session, partitionRecordMap, startTimeNanos); + //At-least once commit handling (if order is reversed it is at-most once) + session.commit(); + commitOffsets(lease, partitionRecordMap); + } catch (final KafkaException ke) { + lease.poison(); + getLogger().error("Problem while accessing kafka consumer " + ke, ke); + context.yield(); + session.rollback(); + } + } + } + + private void commitOffsets(final ConsumerLease lease, final Map>> partitionRecordMap) { + final Map partOffsetMap = new HashMap<>(); + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .forEach((entry) -> { + long maxOffset = entry.getValue().stream() + .mapToLong(record -> record.offset()) + .max() + .getAsLong(); + partOffsetMap.put(entry.getKey(), new OffsetAndMetadata(maxOffset + 1L)); + }); + lease.commitOffsets(partOffsetMap); + } + + private void writeSessionData( + final ProcessContext context, final ProcessSession session, + final Map>> partitionRecordMap, + final long startTimeNanos) { + if (demarcatorBytes != null) { + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .forEach(entry -> { + writeData(context, session, entry.getValue(), startTimeNanos); + }); + } else { + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .flatMap(entry -> entry.getValue().stream()) + .forEach(record -> { + writeData(context, session, Collections.singletonList(record), startTimeNanos); + }); + } + } + + private void writeData(final ProcessContext context, final ProcessSession session, final List> records, final long startTimeNanos) { + final ConsumerRecord firstRecord = records.get(0); + final String offset = String.valueOf(firstRecord.offset()); + final String keyHex = (firstRecord.key() != null) ? DatatypeConverter.printHexBinary(firstRecord.key()) : null; + final String topic = firstRecord.topic(); + final String partition = String.valueOf(firstRecord.partition()); + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + boolean useDemarcator = false; + for (final ConsumerRecord record : records) { + if (useDemarcator) { + out.write(demarcatorBytes); + } + out.write(record.value()); + useDemarcator = true; + } + }); + final Map kafkaAttrs = new HashMap<>(); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, offset); + if (keyHex != null && records.size() == 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY_HEX, keyHex); + } + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, partition); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, topic); + if (records.size() > 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(records.size())); + } + flowFile = session.putAllAttributes(flowFile, kafkaAttrs); + final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); + final String transitUri = KafkaProcessorUtils.buildTransitURI( + context.getProperty(SECURITY_PROTOCOL).getValue(), + context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(), + topic); + session.getProvenanceReporter().receive(flowFile, transitUri, executionDurationMillis); + this.getLogger().debug("Created {} containing {} messages from Kafka topic {}, partition {}, starting offset {} in {} millis", + new Object[]{flowFile, records.size(), topic, partition, offset, executionDurationMillis}); + session.transfer(flowFile, REL_SUCCESS); + } + + /** + * Populates the given partitionRecordMap with new records until we poll + * that returns no records or until we have enough data. It is important to + * ensure we keep items grouped by their topic and partition so that when we + * bundle them we bundle them intelligently and so that we can set offsets + * properly even across multiple poll calls. + */ + private boolean gatherDataFromKafka(final ConsumerLease lease, final Map>> partitionRecordMap, ProcessContext context) { + final long startNanos = System.nanoTime(); + boolean foundData = false; + ConsumerRecords records; + final int maxRecords = context.getProperty(MAX_POLL_RECORDS).asInteger(); + do { + records = lease.poll(); + + for (final TopicPartition partition : records.partitions()) { + List> currList = partitionRecordMap.get(partition); + if (currList == null) { + currList = new ArrayList<>(); + partitionRecordMap.put(partition, currList); + } + currList.addAll(records.records(partition)); + if (currList.size() > 0) { + foundData = true; + } + } + //If we received data and we still want to get more + } while (!records.isEmpty() && !checkIfGatheredEnoughData(partitionRecordMap, maxRecords, startNanos)); + return foundData; + } + + /** + * Determines if we have enough data as-is and should move on. + * + * @return true if we've been gathering for more than 500 ms or if we're + * demarcating and have more than 50 flowfiles worth or if we're per message + * and have more than 2000 flowfiles or if totalMessageSize is greater than + * two megabytes; false otherwise + * + * Implementation note: 500 millis and 5 MB are magic numbers. These may + * need to be tuned. They get at how often offsets will get committed to + * kafka relative to how many records will get buffered into memory in a + * poll call before writing to repos. + */ + private boolean checkIfGatheredEnoughData(final Map>> partitionRecordMap, final int maxRecords, final long startTimeNanos) { + + final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); + + if (durationMillis > 500) { + return true; + } + + int topicPartitionsFilled = 0; + int totalRecords = 0; + long totalRecordSize = 0; + + for (final List> recordList : partitionRecordMap.values()) { + if (!recordList.isEmpty()) { + topicPartitionsFilled++; + } + totalRecords += recordList.size(); + for (final ConsumerRecord rec : recordList) { + totalRecordSize += rec.value().length; + } + } + + if (demarcatorBytes != null && demarcatorBytes.length > 0) { + return topicPartitionsFilled > 50; + } else if (totalRecordSize > FIVE_MB) { + return true; + } else { + return totalRecords > maxRecords; + } + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java new file mode 100644 index 0000000000..b954eba344 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.io.Closeable; +import java.util.Map; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +/** + * This class represents a lease to access a Kafka Consumer object. The lease is + * intended to be obtained from a ConsumerPool. The lease is closeable to allow + * for the clean model of a try w/resources whereby non-exceptional cases mean + * the lease will be returned to the pool for future use by others. A given + * lease may only belong to a single thread a time. + */ +public interface ConsumerLease extends Closeable { + + /** + * Executes a poll on the underlying Kafka Consumer. + * + * @return ConsumerRecords retrieved in the poll. + * @throws KafkaException if issue occurs talking to underlying resource. + */ + ConsumerRecords poll() throws KafkaException; + + /** + * Notifies Kafka to commit the offsets for the specified topic/partition + * pairs to the specified offsets w/the given metadata. This can offer + * higher performance than the other commitOffsets call as it allows the + * kafka client to collect more data from Kafka before committing the + * offsets. + * + * @param offsets offsets + * @throws KafkaException if issue occurs talking to underlying resource. + */ + void commitOffsets(Map offsets) throws KafkaException; + + /** + * Notifies that this lease is poisoned and should not be reused. + */ + void poison(); + + /** + * Notifies that this lease is to be returned. The pool may optionally reuse + * this lease with another client. No further references by the caller + * should occur after calling close. + */ + @Override + void close(); + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java new file mode 100644 index 0000000000..3f20b8f4cb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.nifi.logging.ComponentLog; + +import java.io.Closeable; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +/** + * A pool of Kafka Consumers for a given topic. Consumers can be obtained by + * calling 'obtainConsumer'. Once closed the pool is ready to be immediately + * used again. + */ +public class ConsumerPool implements Closeable { + + private final AtomicInteger activeLeaseCount = new AtomicInteger(0); + private final int maxLeases; + private final Queue consumerLeases; + private final List topics; + private final Map kafkaProperties; + private final ComponentLog logger; + + private final AtomicLong consumerCreatedCountRef = new AtomicLong(); + private final AtomicLong consumerClosedCountRef = new AtomicLong(); + private final AtomicLong leasesObtainedCountRef = new AtomicLong(); + private final AtomicLong productivePollCountRef = new AtomicLong(); + private final AtomicLong unproductivePollCountRef = new AtomicLong(); + + /** + * Creates a pool of KafkaConsumer objects that will grow up to the maximum + * indicated leases. Consumers are lazily initialized. + * + * @param maxLeases maximum number of active leases in the pool + * @param topics the topics to consume from + * @param kafkaProperties the properties for each consumer + * @param logger the logger to report any errors/warnings + */ + public ConsumerPool(final int maxLeases, final List topics, final Map kafkaProperties, final ComponentLog logger) { + this.maxLeases = maxLeases; + if (maxLeases <= 0) { + throw new IllegalArgumentException("Max leases value must be greather than zero."); + } + this.logger = logger; + if (topics == null || topics.isEmpty()) { + throw new IllegalArgumentException("Must have a list of one or more topics"); + } + this.topics = topics; + this.kafkaProperties = new HashMap<>(kafkaProperties); + this.consumerLeases = new ArrayDeque<>(); + } + + /** + * Obtains a consumer from the pool if one is available + * + * @return consumer from the pool + * @throws IllegalArgumentException if pool already contains + */ + public ConsumerLease obtainConsumer() { + final ConsumerLease lease; + final int activeLeases; + synchronized (this) { + lease = consumerLeases.poll(); + activeLeases = activeLeaseCount.get(); + } + if (lease == null && activeLeases >= maxLeases) { + logger.warn("No available consumers and cannot create any as max consumer leases limit reached - verify pool settings"); + return null; + } + leasesObtainedCountRef.incrementAndGet(); + return (lease == null) ? createConsumer() : lease; + } + + protected Consumer createKafkaConsumer() { + return new KafkaConsumer<>(kafkaProperties); + } + + private ConsumerLease createConsumer() { + final Consumer kafkaConsumer = createKafkaConsumer(); + consumerCreatedCountRef.incrementAndGet(); + try { + kafkaConsumer.subscribe(topics); + } catch (final KafkaException kex) { + try { + kafkaConsumer.close(); + consumerClosedCountRef.incrementAndGet(); + } catch (final Exception ex) { + consumerClosedCountRef.incrementAndGet(); + //ignore + } + throw kex; + } + + final ConsumerLease lease = new ConsumerLease() { + + private volatile boolean poisoned = false; + private volatile boolean closed = false; + + @Override + public ConsumerRecords poll() { + + if (poisoned) { + throw new KafkaException("The consumer is poisoned and should no longer be used"); + } + + try { + final ConsumerRecords records = kafkaConsumer.poll(50); + if (records.isEmpty()) { + unproductivePollCountRef.incrementAndGet(); + } else { + productivePollCountRef.incrementAndGet(); + } + return records; + } catch (final KafkaException kex) { + logger.warn("Unable to poll from Kafka consumer so will poison and close this " + kafkaConsumer, kex); + poison(); + close(); + throw kex; + } + } + + @Override + public void commitOffsets(final Map offsets) { + + if (poisoned) { + throw new KafkaException("The consumer is poisoned and should no longer be used"); + } + try { + kafkaConsumer.commitSync(offsets); + } catch (final KafkaException kex) { + logger.warn("Unable to commit kafka consumer offsets so will poison and close this " + kafkaConsumer, kex); + poison(); + close(); + throw kex; + } + } + + @Override + public void close() { + if (closed) { + return; + } + if (poisoned || activeLeaseCount.get() > maxLeases) { + closeConsumer(kafkaConsumer); + activeLeaseCount.decrementAndGet(); + closed = true; + } else { + final boolean added; + synchronized (ConsumerPool.this) { + added = consumerLeases.offer(this); + } + if (!added) { + closeConsumer(kafkaConsumer); + activeLeaseCount.decrementAndGet(); + } + } + } + + @Override + public void poison() { + poisoned = true; + } + }; + activeLeaseCount.incrementAndGet(); + return lease; + } + + /** + * Closes all consumers in the pool. Can be safely recalled. + */ + @Override + public void close() { + final List leases = new ArrayList<>(); + synchronized (this) { + ConsumerLease lease = null; + while ((lease = consumerLeases.poll()) != null) { + leases.add(lease); + } + } + for (final ConsumerLease lease : leases) { + lease.poison(); + lease.close(); + } + } + + private void closeConsumer(final Consumer consumer) { + try { + consumer.unsubscribe(); + } catch (Exception e) { + logger.warn("Failed while unsubscribing " + consumer, e); + } + + try { + consumer.close(); + consumerClosedCountRef.incrementAndGet(); + } catch (Exception e) { + consumerClosedCountRef.incrementAndGet(); + logger.warn("Failed while closing " + consumer, e); + } + } + + PoolStats getPoolStats() { + return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get(), productivePollCountRef.get(), unproductivePollCountRef.get()); + } + + static final class PoolStats { + + final long consumerCreatedCount; + final long consumerClosedCount; + final long leasesObtainedCount; + final long productivePollCount; + final long unproductivePollCount; + + PoolStats( + final long consumerCreatedCount, + final long consumerClosedCount, + final long leasesObtainedCount, + final long productivePollCount, + final long unproductivePollCount + ) { + this.consumerCreatedCount = consumerCreatedCount; + this.consumerClosedCount = consumerClosedCount; + this.leasesObtainedCount = leasesObtainedCount; + this.productivePollCount = productivePollCount; + this.unproductivePollCount = unproductivePollCount; + } + + @Override + public String toString() { + return "Created Consumers [" + consumerCreatedCount + "]\n" + + "Closed Consumers [" + consumerClosedCount + "]\n" + + "Leases Obtained [" + leasesObtainedCount + "]\n" + + "Productive Polls [" + productivePollCount + "]\n" + + "Unproductive Polls [" + unproductivePollCount + "]\n"; + } + + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java new file mode 100644 index 0000000000..c2cc32a5df --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; +import org.apache.kafka.clients.CommonClientConfigs; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class KafkaProcessorUtils { + + final Logger logger = LoggerFactory.getLogger(this.getClass()); + + private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}"; + + private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*"; + + static final Pattern HEX_KEY_PATTERN = Pattern.compile("(?:[0123456789abcdefABCDEF]{2})+"); + + static final String KAFKA_KEY_HEX = "kafka.key.hex"; + static final String KAFKA_TOPIC = "kafka.topic"; + static final String KAFKA_PARTITION = "kafka.partition"; + static final String KAFKA_OFFSET = "kafka.offset"; + static final String KAFKA_COUNT = "kafka.count"; + static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT"); + static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL"); + static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT"); + static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL"); + + static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder() + .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) + .displayName("Kafka Brokers") + .description("A comma-separated list of known Kafka Brokers in the format :") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) + .expressionLanguageSupported(true) + .defaultValue("localhost:9092") + .build(); + static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder() + .name("security.protocol") + .displayName("Security Protocol") + .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + .required(true) + .expressionLanguageSupported(false) + .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL) + .defaultValue(SEC_PLAINTEXT.getValue()) + .build(); + static final PropertyDescriptor KERBEROS_PRINCIPLE = new PropertyDescriptor.Builder() + .name("sasl.kerberos.service.name") + .displayName("Kerberos Service Name") + .description("The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config. " + + "Corresponds to Kafka's 'security.protocol' property." + + "It is ignored unless one of the SASL options of the are selected.") + .required(false) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("ssl.context.service") + .displayName("SSL Context Service") + .description("Specifies the SSL Context Service to use for communicating with Kafka.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + + static List getCommonPropertyDescriptors() { + return Arrays.asList( + BOOTSTRAP_SERVERS, + SECURITY_PROTOCOL, + KERBEROS_PRINCIPLE, + SSL_CONTEXT_SERVICE + ); + } + + static Collection validateCommonProperties(final ValidationContext validationContext) { + List results = new ArrayList<>(); + + String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue(); + + /* + * validates that if one of SASL (Kerberos) option is selected for + * security protocol, then Kerberos principal is provided as well + */ + if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) { + String kerberosPrincipal = validationContext.getProperty(KERBEROS_PRINCIPLE).getValue(); + if (kerberosPrincipal == null || kerberosPrincipal.trim().length() == 0) { + results.add(new ValidationResult.Builder().subject(KERBEROS_PRINCIPLE.getDisplayName()).valid(false) + .explanation("The <" + KERBEROS_PRINCIPLE.getDisplayName() + "> property must be set when <" + + SECURITY_PROTOCOL.getDisplayName() + "> is configured as '" + + SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.") + .build()); + } + } + + //If SSL or SASL_SSL then CS must be set. + final boolean sslProtocol = SEC_SSL.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol); + final boolean csSet = validationContext.getProperty(SSL_CONTEXT_SERVICE).isSet(); + if (csSet && !sslProtocol) { + results.add(new ValidationResult.Builder().subject(SECURITY_PROTOCOL.getDisplayName()).valid(false) + .explanation("If you set the SSL Controller Service you should also choose an SSL based security protocol.").build()); + } + if (!csSet && sslProtocol) { + results.add(new ValidationResult.Builder().subject(SSL_CONTEXT_SERVICE.getDisplayName()).valid(false) + .explanation("If you set to an SSL based protocol you need to set the SSL Controller Service").build()); + } + + final String enableAutoCommit = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).build()).getValue(); + if (enableAutoCommit != null && !enableAutoCommit.toLowerCase().equals("false")) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + .explanation("Enable auto commit must be false. It is managed by the processor.").build()); + } + + final String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()).getValue(); + if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) { + results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG) + .explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build()); + } + + final String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()).getValue(); + if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) { + results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG) + .explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build()); + } + + final String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()).getValue(); + if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG) + .explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build()); + } + + final String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()).getValue(); + if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG) + .explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build()); + } + + return results; + } + + static final class KafkaConfigValidator implements Validator { + + final Class classType; + + public KafkaConfigValidator(final Class classType) { + this.classType = classType; + } + + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + final boolean knownValue = KafkaProcessorUtils.isStaticStringFieldNamePresent(subject, classType, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class); + return new ValidationResult.Builder().subject(subject).explanation("Must be a known configuration parameter for this kafka client").valid(knownValue).build(); + } + }; + + /** + * Builds transit URI for provenance event. The transit URI will be in the + * form of <security.protocol>://<bootstrap.servers>/topic + */ + static String buildTransitURI(String securityProtocol, String brokers, String topic) { + StringBuilder builder = new StringBuilder(); + builder.append(securityProtocol); + builder.append("://"); + builder.append(brokers); + builder.append("/"); + builder.append(topic); + return builder.toString(); + } + + static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map mapToPopulate) { + for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) { + if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) { + // Translate SSLContext Service configuration into Kafka properties + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + if (sslContextService != null && sslContextService.isKeyStoreConfigured()) { + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile()); + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword()); + final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword(); + mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass); + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType()); + } + + if (sslContextService != null && sslContextService.isTrustStoreConfigured()) { + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile()); + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword()); + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType()); + } + } + String pName = propertyDescriptor.getName(); + String pValue = propertyDescriptor.isExpressionLanguageSupported() + ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue() + : context.getProperty(propertyDescriptor).getValue(); + if (pValue != null) { + if (pName.endsWith(".ms")) { // kafka standard time notation + pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS)); + } + if (isStaticStringFieldNamePresent(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) { + mapToPopulate.put(pName, pValue); + } + } + } + } + + private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) { + return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name); + } + + private static Set getPublicStaticStringFieldValues(final Class... classes) { + final Set strings = new HashSet<>(); + for (final Class classType : classes) { + for (final Field field : classType.getDeclaredFields()) { + if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) { + try { + strings.add(String.valueOf(field.get(null))); + } catch (IllegalArgumentException | IllegalAccessException ex) { + //ignore + } + } + } + } + return strings; + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java similarity index 92% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java index f684bfa3cf..31a084f133 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java @@ -39,6 +39,7 @@ import org.apache.nifi.stream.io.util.StreamDemarcator; * with sending contents of the {@link FlowFile}s to Kafka. */ class KafkaPublisher implements Closeable { + private final Producer kafkaProducer; private volatile long ackWaitTime = 30000; @@ -56,9 +57,8 @@ class KafkaPublisher implements Closeable { * corresponding Kafka {@link KafkaProducer} using provided Kafka * configuration properties. * - * @param kafkaProperties - * instance of {@link Properties} used to bootstrap - * {@link KafkaProducer} + * @param kafkaProperties instance of {@link Properties} used to bootstrap + * {@link KafkaProducer} */ KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) { this.kafkaProducer = new KafkaProducer<>(kafkaProperties); @@ -89,9 +89,8 @@ class KafkaPublisher implements Closeable { * index of the last ACKed message, so upon retry only messages with the * higher index are sent. * - * @param publishingContext - * instance of {@link PublishingContext} which hold context - * information about the message(s) to be sent. + * @param publishingContext instance of {@link PublishingContext} which hold + * context information about the message(s) to be sent. * @return The index of the last successful offset. */ KafkaPublisherResult publish(PublishingContext publishingContext) { @@ -110,7 +109,7 @@ class KafkaPublisher implements Closeable { ProducerRecord message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes); resultFutures.add(this.kafkaProducer.send(message)); - if (tokenCounter % this.ackCheckSize == 0){ + if (tokenCounter % this.ackCheckSize == 0) { int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex); resultFutures.clear(); if (lastAckedMessageIndex % this.ackCheckSize != 0) { @@ -154,14 +153,12 @@ class KafkaPublisher implements Closeable { * be considered non-delivered and therefore could be resent at the later * time. * - * @param sendFutures - * list of {@link Future}s representing results of publishing to - * Kafka + * @param sendFutures list of {@link Future}s representing results of + * publishing to Kafka * - * @param lastAckMessageIndex - * the index of the last ACKed message. It is important to - * provide the last ACKed message especially while re-trying so - * the proper index is maintained. + * @param lastAckMessageIndex the index of the last ACKed message. It is + * important to provide the last ACKed message especially while re-trying so + * the proper index is maintained. */ private int processAcks(List> sendFutures, int lastAckMessageIndex) { boolean exceptionThrown = false; @@ -210,8 +207,10 @@ class KafkaPublisher implements Closeable { * Encapsulates the result received from publishing messages to Kafka */ static class KafkaPublisherResult { + private final int messagesSent; private final int lastMessageAcked; + KafkaPublisherResult(int messagesSent, int lastMessageAcked) { this.messagesSent = messagesSent; this.lastMessageAcked = lastMessageAcked; diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java similarity index 99% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java index 8c948df395..64ab4ce9c8 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java @@ -34,6 +34,7 @@ final public class Partitioners { * distributes load between all available partitions. */ public static class RoundRobinPartitioner implements Partitioner { + private volatile int index; @Override diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java new file mode 100644 index 0000000000..e29f2af850 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java @@ -0,0 +1,516 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import javax.xml.bind.DatatypeConverter; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractSessionFactoryProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.10"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.10 producer. " + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring" + + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can. In the mean time" + + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.", + description = "These properties will be added on the Kafka configuration after loading any provided configuration properties." + + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged." + + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ") +public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor { + + private final Logger logger = LoggerFactory.getLogger(this.getClass()); + + protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id"; + + protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx"; + + protected static final String FAILED_TOPIC_ATTR = "failed.topic"; + + protected static final String FAILED_KEY_ATTR = "failed.key"; + + protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter"; + + protected static final String MSG_COUNT = "msg.count"; + + static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery", + "FlowFile will be routed to failure unless the message is replicated to the appropriate " + + "number of Kafka Nodes according to the Topic configuration"); + static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery", + "FlowFile will be routed to success if the message is received by a single Kafka node, " + + "whether or not it is replicated. This is faster than " + + "but can result in data loss if a Kafka node crashes"); + static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort", + "FlowFile will be routed to success after successfully writing the content to a Kafka node, " + + "without waiting for a response. This provides the best performance but may result in data loss."); + + static final AllowableValue ROUND_ROBIN_PARTITIONING = new AllowableValue(Partitioners.RoundRobinPartitioner.class.getName(), + Partitioners.RoundRobinPartitioner.class.getSimpleName(), + "Messages will be assigned partitions in a round-robin fashion, sending the first message to Partition 1, " + + "the next Partition to Partition 2, and so on, wrapping as necessary."); + static final AllowableValue RANDOM_PARTITIONING = new AllowableValue("org.apache.kafka.clients.producer.internals.DefaultPartitioner", + "DefaultPartitioner", "Messages will be assigned to random partitions."); + + static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Kafka Topic to publish to.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder() + .name(ProducerConfig.ACKS_CONFIG) + .displayName("Delivery Guarantee") + .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.") + .required(true) + .expressionLanguageSupported(false) + .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED) + .defaultValue(DELIVERY_BEST_EFFORT.getValue()) + .build(); + + static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder() + .name(ProducerConfig.MAX_BLOCK_MS_CONFIG) + .displayName("Meta Data Wait Time") + .description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the " + + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("30 sec") + .build(); + + static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() + .name("kafka-key") + .displayName("Kafka Key") + .description("The Key to use for the Message. It will be serialized as UTF-8 bytes. " + + "If not specified then the flow file attribute kafka.key.hex is used if present " + + "and we're not demarcating. In that case the hex string is coverted to its byte" + + "form and written as a byte[] key.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder() + .name("message-demarcator") + .displayName("Message Demarcator") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within " + + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the " + + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. " + + "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on your OS.") + .build(); + + static final PropertyDescriptor PARTITION_CLASS = new PropertyDescriptor.Builder() + .name(ProducerConfig.PARTITIONER_CLASS_CONFIG) + .displayName("Partitioner class") + .description("Specifies which class to use to compute a partition id for a message. Corresponds to Kafka's 'partitioner.class' property.") + .allowableValues(ROUND_ROBIN_PARTITIONING, RANDOM_PARTITIONING) + .defaultValue(RANDOM_PARTITIONING.getValue()) + .required(false) + .build(); + + static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder() + .name(ProducerConfig.COMPRESSION_TYPE_CONFIG) + .displayName("Compression Type") + .description("This parameter allows you to specify the compression codec for all data generated by this producer.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .allowableValues("none", "gzip", "snappy", "lz4") + .defaultValue("none") + .build(); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles for which all content was sent to Kafka.") + .build(); + + static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship") + .build(); + + static final List DESCRIPTORS; + + static final Set RELATIONSHIPS; + + private volatile String brokers; + + private final AtomicInteger taskCounter = new AtomicInteger(); + + private volatile boolean acceptTask = true; + + /* + * Will ensure that list of PropertyDescriptors is build only once, since + * all other lifecycle methods are invoked multiple times. + */ + static { + final List _descriptors = new ArrayList<>(); + _descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + _descriptors.add(TOPIC); + _descriptors.add(DELIVERY_GUARANTEE); + _descriptors.add(KEY); + _descriptors.add(MESSAGE_DEMARCATOR); + _descriptors.add(META_WAIT_TIME); + _descriptors.add(PARTITION_CLASS); + _descriptors.add(COMPRESSION_CODEC); + + DESCRIPTORS = Collections.unmodifiableList(_descriptors); + + final Set _relationships = new HashSet<>(); + _relationships.add(REL_SUCCESS); + _relationships.add(REL_FAILURE); + RELATIONSHIPS = Collections.unmodifiableSet(_relationships); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.") + .name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + return KafkaProcessorUtils.validateCommonProperties(validationContext); + } + + volatile KafkaPublisher kafkaPublisher; + + /** + * This thread-safe operation will delegate to + * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first + * checking and creating (if necessary) Kafka resource which could be either + * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and + * destroy the underlying Kafka resource upon catching an {@link Exception} + * raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}. + * After Kafka resource is destroyed it will be re-created upon the next + * invocation of this operation essentially providing a self healing + * mechanism to deal with potentially corrupted resource. + *

+ * Keep in mind that upon catching an exception the state of this processor + * will be set to no longer accept any more tasks, until Kafka resource is + * reset. This means that in a multi-threaded situation currently executing + * tasks will be given a chance to complete while no new tasks will be + * accepted. + * + * @param context context + * @param sessionFactory factory + */ + @Override + public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException { + if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted. + this.taskCounter.incrementAndGet(); + final ProcessSession session = sessionFactory.createSession(); + try { + /* + * We can't be doing double null check here since as a pattern + * it only works for lazy init but not reset, which is what we + * are doing here. In fact the first null check is dangerous + * since 'kafkaPublisher' can become null right after its null + * check passed causing subsequent NPE. + */ + synchronized (this) { + if (this.kafkaPublisher == null) { + this.kafkaPublisher = this.buildKafkaResource(context, session); + } + } + + /* + * The 'processed' boolean flag does not imply any failure or success. It simply states that: + * - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated + * - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile + */ + boolean processed = this.rendezvousWithKafka(context, session); + session.commit(); + if (!processed) { + context.yield(); + } + } catch (Throwable e) { + this.acceptTask = false; + session.rollback(true); + this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e}); + } finally { + synchronized (this) { + if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) { + this.close(); + this.acceptTask = true; + } + } + } + } else { + this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); + this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); + context.yield(); + } + } + + /** + * Will call {@link Closeable#close()} on the target resource after which + * the target resource will be set to null. Should only be called when there + * are no more threads being executed on this processor or when it has been + * verified that only a single thread remains. + * + * @see KafkaPublisher + * @see KafkaConsumer + */ + @OnStopped + public void close() { + try { + if (this.kafkaPublisher != null) { + try { + this.kafkaPublisher.close(); + } catch (Exception e) { + this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e); + } + } + } finally { + this.kafkaPublisher = null; + } + } + + /** + * Will rendezvous with Kafka if {@link ProcessSession} contains + * {@link FlowFile} producing a result {@link FlowFile}. + *
+ * The result {@link FlowFile} that is successful is then transfered to + * {@link #REL_SUCCESS} + *
+ * The result {@link FlowFile} that is failed is then transfered to + * {@link #REL_FAILURE} + * + */ + protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) { + FlowFile flowFile = session.get(); + if (flowFile != null) { + long start = System.nanoTime(); + flowFile = this.doRendezvousWithKafka(flowFile, context, session); + Relationship relationship = REL_SUCCESS; + if (!this.isFailedFlowFile(flowFile)) { + String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic); + session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration); + this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis", + new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration}); + } else { + relationship = REL_FAILURE; + flowFile = session.penalize(flowFile); + } + session.transfer(flowFile, relationship); + } + return flowFile != null; + } + + /** + * Builds and instance of {@link KafkaPublisher}. + */ + protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) { + final Map kafkaProps = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps); + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + final Properties props = new Properties(); + props.putAll(kafkaProps); + KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger()); + return publisher; + } + + /** + * Will rendezvous with {@link KafkaPublisher} after building + * {@link PublishingContext} and will produce the resulting + * {@link FlowFile}. The resulting FlowFile contains all required + * information to determine if message publishing originated from the + * provided FlowFile has actually succeeded fully, partially or failed + * completely (see {@link #isFailedFlowFile(FlowFile)}. + */ + private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) { + final AtomicReference publishResultRef = new AtomicReference<>(); + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(InputStream contentStream) throws IOException { + PublishingContext publishingContext = PublishKafka_0_10.this.buildPublishingContext(flowFile, context, contentStream); + KafkaPublisher.KafkaPublisherResult result = PublishKafka_0_10.this.kafkaPublisher.publish(publishingContext); + publishResultRef.set(result); + } + }); + + FlowFile resultFile = publishResultRef.get().isAllAcked() + ? this.cleanUpFlowFileIfNecessary(flowFile, session) + : session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context)); + + if (!this.isFailedFlowFile(resultFile)) { + resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent())); + } + return resultFile; + } + + /** + * Builds {@link PublishingContext} for message(s) to be sent to Kafka. + * {@link PublishingContext} contains all contextual information required by + * {@link KafkaPublisher} to publish to Kafka. Such information contains + * things like topic name, content stream, delimiter, key and last ACKed + * message for cases where provided FlowFile is being retried (failed in the + * past). + *
+ * For the clean FlowFile (file that has been sent for the first time), + * PublishingContext will be built form {@link ProcessContext} associated + * with this invocation. + *
+ * For the failed FlowFile, {@link PublishingContext} will be built from + * attributes of that FlowFile which by then will already contain required + * information (e.g., topic, key, delimiter etc.). This is required to + * ensure the affinity of the retry in the even where processor + * configuration has changed. However keep in mind that failed FlowFile is + * only considered a failed FlowFile if it is being re-processed by the same + * processor (determined via {@link #FAILED_PROC_ID_ATTR}, see + * {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to + * another PublishKafka0_10 processor it is treated as a fresh FlowFile + * regardless if it has #FAILED* attributes set. + */ + private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) { + String topicName; + byte[] keyBytes; + byte[] delimiterBytes = null; + int lastAckedMessageIndex = -1; + if (this.isFailedFlowFile(flowFile)) { + lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX)); + topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR); + keyBytes = flowFile.getAttribute(FAILED_KEY_ATTR) != null + ? flowFile.getAttribute(FAILED_KEY_ATTR).getBytes(StandardCharsets.UTF_8) : null; + delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null + ? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null; + + } else { + topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); + keyBytes = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8); + String keyHex = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_KEY_HEX); + if (_key == null && keyHex != null && KafkaProcessorUtils.HEX_KEY_PATTERN.matcher(keyHex).matches()) { + keyBytes = DatatypeConverter.parseHexBinary(keyHex); + } + delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR) + .evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null; + } + + PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex); + publishingContext.setKeyBytes(keyBytes); + publishingContext.setDelimiterBytes(delimiterBytes); + return publishingContext; + } + + /** + * Will remove FAILED_* attributes if FlowFile is no longer considered a + * failed FlowFile + * + * @see #isFailedFlowFile(FlowFile) + */ + private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) { + if (this.isFailedFlowFile(flowFile)) { + Set keysToRemove = new HashSet<>(); + keysToRemove.add(FAILED_DELIMITER_ATTR); + keysToRemove.add(FAILED_KEY_ATTR); + keysToRemove.add(FAILED_TOPIC_ATTR); + keysToRemove.add(FAILED_PROC_ID_ATTR); + keysToRemove.add(FAILED_LAST_ACK_IDX); + flowFile = session.removeAllAttributes(flowFile, keysToRemove); + } + return flowFile; + } + + /** + * Builds a {@link Map} of FAILED_* attributes + * + * @see #FAILED_PROC_ID_ATTR + * @see #FAILED_LAST_ACK_IDX + * @see #FAILED_TOPIC_ATTR + * @see #FAILED_KEY_ATTR + * @see #FAILED_DELIMITER_ATTR + */ + private Map buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile, ProcessContext context) { + Map attributes = new HashMap<>(); + attributes.put(FAILED_PROC_ID_ATTR, this.getIdentifier()); + attributes.put(FAILED_LAST_ACK_IDX, String.valueOf(lastAckedMessageIndex)); + attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue()); + attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue()); + attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet() + ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null); + return attributes; + } + + /** + * Returns 'true' if provided FlowFile is a failed FlowFile. A failed + * FlowFile contains {@link #FAILED_PROC_ID_ATTR}. + */ + private boolean isFailedFlowFile(FlowFile flowFile) { + return this.getIdentifier().equals(flowFile.getAttribute(FAILED_PROC_ID_ATTR)); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..aa1d4e2a22 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.processors.kafka.pubsub.PublishKafka_0_10 +org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html new file mode 100644 index 0000000000..2ce6b515f9 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html @@ -0,0 +1,33 @@ + + + + + + ConsumeKafka + + + + + +

Description:

+

+ This Processors polls Apache Kafka + for data using KafkaConsumer API available with Kafka 0.10+. When a message is received + from Kafka, this Processor emits a FlowFile where the content of the FlowFile is the value + of the Kafka message. +

+ + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html new file mode 100644 index 0000000000..dfd92b32ed --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html @@ -0,0 +1,47 @@ + + + + + + PublishKafka + + + + + +

Description:

+

+ This Processors puts the contents of a FlowFile to a Topic in + Apache Kafka using KafkaProducer API available + with Kafka 0.10+ API. The content of a FlowFile becomes the contents of a Kafka message. + This message is optionally assigned a key by using the <Kafka Key> Property. +

+ +

+ The Processor allows the user to configure an optional Message Demarcator that + can be used to send many messages per FlowFile. For example, a \n could be used + to indicate that the contents of the FlowFile should be used to send one message + per line of text. It also supports multi-char demarcators (e.g., 'my custom demarcator'). + If the property is not set, the entire contents of the FlowFile + will be sent as a single message. When using the demarcator, if some messages are + successfully sent but other messages fail to send, the resulting FlowFile will be + considered a failed FlowFuile and will have additional attributes to that effect. + One of such attributes is 'failed.last.idx' which indicates the index of the last message + that was successfully ACKed by Kafka. (if no demarcator is used the value of this index will be -1). + This will allow PublishKafka to only re-send un-ACKed messages on the next re-try. +

+ + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java new file mode 100644 index 0000000000..c172b0316e --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java @@ -0,0 +1,496 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.UUID; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class ConsumeKafkaTest { + + static class MockConsumerPool extends ConsumerPool { + + final int actualMaxLeases; + final List actualTopics; + final Map actualKafkaProperties; + boolean throwKafkaExceptionOnPoll = false; + boolean throwKafkaExceptionOnCommit = false; + Queue> nextPlannedRecordsQueue = new ArrayDeque<>(); + Map nextExpectedCommitOffsets = null; + Map actualCommitOffsets = null; + boolean wasConsumerLeasePoisoned = false; + boolean wasConsumerLeaseClosed = false; + boolean wasPoolClosed = false; + + public MockConsumerPool(int maxLeases, List topics, Map kafkaProperties, ComponentLog logger) { + super(maxLeases, topics, kafkaProperties, null); + actualMaxLeases = maxLeases; + actualTopics = topics; + actualKafkaProperties = kafkaProperties; + } + + @Override + public ConsumerLease obtainConsumer() { + return new ConsumerLease() { + @Override + public ConsumerRecords poll() { + if (throwKafkaExceptionOnPoll) { + throw new KafkaException("i planned to fail"); + } + final ConsumerRecords records = nextPlannedRecordsQueue.poll(); + return (records == null) ? ConsumerRecords.empty() : records; + } + + @Override + public void commitOffsets(Map offsets) { + if (throwKafkaExceptionOnCommit) { + throw new KafkaException("i planned to fail"); + } + actualCommitOffsets = offsets; + } + + @Override + public void poison() { + wasConsumerLeasePoisoned = true; + } + + @Override + public void close() { + wasConsumerLeaseClosed = true; + } + }; + } + + @Override + public void close() { + wasPoolClosed = true; + } + + void resetState() { + throwKafkaExceptionOnPoll = false; + throwKafkaExceptionOnCommit = false; + nextPlannedRecordsQueue = null; + nextExpectedCommitOffsets = null; + wasConsumerLeasePoisoned = false; + wasConsumerLeaseClosed = false; + wasPoolClosed = false; + } + + } + + @Test + public void validateCustomValidatorSettings() throws Exception { + ConsumeKafka_0_10 consumeKafka = new ConsumeKafka_0_10(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + runner.assertValid(); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo"); + runner.assertNotValid(); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + runner.assertValid(); + runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + runner.assertValid(); + runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + runner.assertNotValid(); + } + + @Test + public void validatePropertiesValidation() throws Exception { + ConsumeKafka_0_10 consumeKafka = new ConsumeKafka_0_10(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + + runner.removeProperty(ConsumeKafka_0_10.GROUP_ID); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("invalid because group.id is required")); + } + + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, ""); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); + } + + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, " "); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); + } + } + + @Test + public void validateGetAllMessages() throws Exception { + String groupName = "validateGetAllMessages"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues); + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(firstRecs); + mockPool.nextPlannedRecordsQueue.add(secondRecs); + + ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS); + + assertEquals(expectedTopics, mockPool.actualTopics); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count()); + + if (mockPool.nextPlannedRecordsQueue.isEmpty()) { + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-5")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-6")).count()); + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset()); + } else { + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + } + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + } + + @Test + public void validateGetLotsOfMessages() throws Exception { + String groupName = "validateGetLotsOfMessages"; + + final byte[][] firstPassValues = new byte[10010][1]; + for (final byte[] value : firstPassValues) { + value[0] = 0x12; + } + final ConsumerRecords firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues); + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(firstRecs); + mockPool.nextPlannedRecordsQueue.add(secondRecs); + + ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS); + + assertEquals(10010, flowFiles.stream().map(ff -> ff.toByteArray()).filter(content -> content.length == 1 && content[0] == 0x12).count()); + assertEquals(1, mockPool.nextPlannedRecordsQueue.size()); + + assertEquals(1, mockPool.actualCommitOffsets.size()); + assertEquals(10011L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + } + + private ConsumerRecords createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) { + final Map>> map = new HashMap<>(); + final TopicPartition tPart = new TopicPartition(topic, partition); + final List> records = new ArrayList<>(); + long offset = startingOffset; + for (final byte[] rawRecord : rawRecords) { + final ConsumerRecord rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord); + records.add(rec); + } + map.put(tPart, records); + return new ConsumerRecords(map); + } + + private ConsumerRecords mergeRecords(final ConsumerRecords... records) { + final Map>> map = new HashMap<>(); + for (final ConsumerRecords rec : records) { + rec.partitions().stream().forEach((part) -> { + final List> conRecs = rec.records(part); + if (map.get(part) != null) { + throw new IllegalStateException("already have that topic/partition in the record map"); + } + map.put(part, conRecs); + }); + } + return new ConsumerRecords<>(map); + } + + @Test + public void validateGetAllMessagesWithProvidedDemarcator() throws Exception { + String groupName = "validateGetAllMessagesWithProvidedDemarcator"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues), + createConsumerRecords("bar", 1, 1L, secondPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + + ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS); + + assertEquals(2, flowFiles.size()); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4blahHello-5blahHello-6")).count()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset()); + } + + @Test + public void validatePollException() throws Exception { + String groupName = "validatePollException"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + mockPool.throwKafkaExceptionOnPoll = true; + + ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, true); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS); + + assertEquals(0, flowFiles.size()); + assertNull(null, mockPool.actualCommitOffsets); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertTrue(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + } + + @Test + public void validateCommitOffsetException() throws Exception { + String groupName = "validateCommitOffsetException"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + mockPool.throwKafkaExceptionOnCommit = true; + + ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, true); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS); + + assertEquals(1, flowFiles.size()); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertTrue(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + assertNull(null, mockPool.actualCommitOffsets); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java new file mode 100644 index 0000000000..7f88ea2b2b --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.util.Collections; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.KafkaException; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processors.kafka.pubsub.ConsumerPool.PoolStats; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConsumerPoolTest { + + Consumer consumer = null; + ComponentLog logger = null; + + @Before + public void setup() { + consumer = mock(Consumer.class); + logger = mock(ComponentLog.class); + } + + @Test + public void validatePoolSimpleCreateClose() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty()); + + try (final ConsumerLease lease = testPool.obtainConsumer()) { + lease.poll(); + lease.commitOffsets(Collections.emptyMap()); + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + assertEquals(1, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } + + @Test + public void validatePoolSimpleBatchCreateClose() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(5, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty()); + + for (int i = 0; i < 100; i++) { + try (final ConsumerLease lease = testPool.obtainConsumer()) { + for (int j = 0; j < 100; j++) { + lease.poll(); + } + lease.commitOffsets(Collections.emptyMap()); + } + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(100, stats.leasesObtainedCount); + assertEquals(10000, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } + + @Test + public void validatePoolConsumerFails() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenThrow(new KafkaException()); + + try (final ConsumerLease lease = testPool.obtainConsumer()) { + lease.poll(); + fail(); + } catch (final KafkaException ke) { + + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + assertEquals(0, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java similarity index 96% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java index 6b8b042d20..19c64af1af 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java @@ -47,6 +47,7 @@ import kafka.consumer.ConsumerIterator; import kafka.consumer.ConsumerTimeoutException; import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; +import org.apache.kafka.clients.producer.ProducerConfig; public class KafkaPublisherTest { @@ -258,9 +259,9 @@ public class KafkaPublisherTest { private Properties buildProducerProperties() { Properties kafkaProperties = new Properties(); - kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName()); - kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName()); - kafkaProperties.setProperty("bootstrap.servers", "localhost:" + kafkaLocal.getKafkaPort()); + kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort()); kafkaProperties.put("auto.create.topics.enable", "true"); return kafkaProperties; } @@ -282,6 +283,7 @@ public class KafkaPublisherTest { } public static class TestPartitioner implements Partitioner { + static int counter; @Override diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java new file mode 100644 index 0000000000..5480ea7dd1 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java @@ -0,0 +1,329 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.nio.charset.StandardCharsets; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; +import org.mockito.Mockito; +import static org.mockito.Mockito.times; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.verify; + +public class PublishKafkaTest { + + @Test + public void validateCustomSerilaizerDeserializerSettings() throws Exception { + PublishKafka_0_10 publishKafka = new PublishKafka_0_10(); + TestRunner runner = TestRunners.newTestRunner(publishKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(PublishKafka_0_10.TOPIC, "foo"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3 sec"); + runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + runner.assertValid(); + runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo"); + runner.assertNotValid(); + } + + @Test + public void validatePropertiesValidation() throws Exception { + PublishKafka_0_10 publishKafka = new PublishKafka_0_10(); + TestRunner runner = TestRunners.newTestRunner(publishKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(PublishKafka_0_10.TOPIC, "foo"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "foo"); + + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("'max.block.ms' validated against 'foo' is invalid")); + } + } + + @Test + public void validateCustomValidation() { + String topicName = "validateCustomValidation"; + PublishKafka_0_10 publishKafka = new PublishKafka_0_10(); + + /* + * Validates that Kerberos principle is required if one of SASL set for + * secirity protocol + */ + TestRunner runner = TestRunners.newTestRunner(publishKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT); + try { + runner.run(); + fail(); + } catch (Throwable e) { + assertTrue(e.getMessage().contains("'Kerberos Service Name' is invalid because")); + } + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateSingleCharacterDemarcatedMessages() { + String topicName = "validateSingleCharacterDemarcatedMessages"; + StubPublishKafka putKafka = new StubPublishKafka(100); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + + runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + verify(producer, times(7)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerA() { + String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner"; + StubPublishKafka putKafka = new StubPublishKafka(100); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName()); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo"); + + runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + verify(producer, times(7)).send(Mockito.any(ProducerRecord.class)); + + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerB() { + String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner"; + StubPublishKafka putKafka = new StubPublishKafka(1); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName()); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo"); + + runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + verify(producer, times(7)).send(Mockito.any(ProducerRecord.class)); + + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateOnSendFailureAndThenResendSuccessA() throws Exception { + String topicName = "validateSendFailureAndThenResendSuccess"; + StubPublishKafka putKafka = new StubPublishKafka(100); + + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3000 millis"); + + final String text = "Hello World\nGoodbye\nfail\n2"; + runner.enqueue(text.getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + verify(producer, times(4)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + putKafka.destroy(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateOnSendFailureAndThenResendSuccessB() throws Exception { + String topicName = "validateSendFailureAndThenResendSuccess"; + StubPublishKafka putKafka = new StubPublishKafka(1); + + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis"); + + final String text = "Hello World\nGoodbye\nfail\n2"; + runner.enqueue(text.getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + verify(producer, times(4)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateOnFutureGetFailureAndThenResendSuccessFirstMessageFail() throws Exception { + String topicName = "validateSendFailureAndThenResendSuccess"; + StubPublishKafka putKafka = new StubPublishKafka(100); + + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis"); + + final String text = "futurefail\nHello World\nGoodbye\n2"; + runner.enqueue(text.getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0); + assertNotNull(ff); + runner.enqueue(ff); + + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + // 6 sends due to duplication + verify(producer, times(5)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateOnFutureGetFailureAndThenResendSuccess() throws Exception { + String topicName = "validateSendFailureAndThenResendSuccess"; + StubPublishKafka putKafka = new StubPublishKafka(100); + + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis"); + + final String text = "Hello World\nGoodbye\nfuturefail\n2"; + runner.enqueue(text.getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0); + assertNotNull(ff); + runner.enqueue(ff); + + runner.run(1, false); + assertEquals(0, runner.getQueueSize().getObjectCount()); + Producer producer = putKafka.getProducer(); + // 6 sends due to duplication + verify(producer, times(6)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateDemarcationIntoEmptyMessages() { + String topicName = "validateDemarcationIntoEmptyMessages"; + StubPublishKafka putKafka = new StubPublishKafka(100); + final TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(PublishKafka_0_10.KEY, "key1"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n"); + + final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8); + runner.enqueue(bytes); + runner.run(1); + Producer producer = putKafka.getProducer(); + verify(producer, times(4)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateComplexRightPartialDemarcatedMessages() { + String topicName = "validateComplexRightPartialDemarcatedMessages"; + StubPublishKafka putKafka = new StubPublishKafka(100); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); + + runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + + Producer producer = putKafka.getProducer(); + verify(producer, times(3)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateComplexLeftPartialDemarcatedMessages() { + String topicName = "validateComplexLeftPartialDemarcatedMessages"; + StubPublishKafka putKafka = new StubPublishKafka(100); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); + + runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>僠<僠WILDSTUFF僠>僠".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1); + Producer producer = putKafka.getProducer(); + verify(producer, times(4)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } + + @SuppressWarnings("unchecked") + @Test + public void validateComplexPartialMatchDemarcatedMessages() { + String topicName = "validateComplexPartialMatchDemarcatedMessages"; + StubPublishKafka putKafka = new StubPublishKafka(100); + TestRunner runner = TestRunners.newTestRunner(putKafka); + runner.setProperty(PublishKafka_0_10.TOPIC, topicName); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); + + runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDBOOMSTUFF僠>僠".getBytes(StandardCharsets.UTF_8)); + runner.run(1, false); + + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1); + Producer producer = putKafka.getProducer(); + verify(producer, times(2)).send(Mockito.any(ProducerRecord.class)); + runner.shutdown(); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java similarity index 93% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java index 5cee6bc182..4a9a1c07ba 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java @@ -72,17 +72,17 @@ public class PublishingContextTest { @Test public void validateOnlyOnceSetPerInstance() { PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic"); - publishingContext.setKeyBytes(new byte[] { 0 }); + publishingContext.setKeyBytes(new byte[]{0}); try { - publishingContext.setKeyBytes(new byte[] { 0 }); + publishingContext.setKeyBytes(new byte[]{0}); fail(); } catch (IllegalArgumentException e) { // success } - publishingContext.setDelimiterBytes(new byte[] { 0 }); + publishingContext.setDelimiterBytes(new byte[]{0}); try { - publishingContext.setDelimiterBytes(new byte[] { 0 }); + publishingContext.setDelimiterBytes(new byte[]{0}); fail(); } catch (IllegalArgumentException e) { // success diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java new file mode 100644 index 0000000000..27d86f5065 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.lang.reflect.Field; +import static org.mockito.Mockito.when; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS; +import org.mockito.Mockito; +import static org.mockito.Mockito.mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class StubPublishKafka extends PublishKafka_0_10 { + + private volatile Producer producer; + + private volatile boolean failed; + + private final int ackCheckSize; + + private final ExecutorService executor = Executors.newCachedThreadPool(); + + StubPublishKafka(int ackCheckSize) { + this.ackCheckSize = ackCheckSize; + } + + public Producer getProducer() { + return producer; + } + + public void destroy() { + this.executor.shutdownNow(); + } + + @SuppressWarnings("unchecked") + @Override + protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) + throws ProcessException { + final Map kafkaProperties = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties); + kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + KafkaPublisher publisher; + try { + Field f = PublishKafka_0_10.class.getDeclaredField("brokers"); + f.setAccessible(true); + f.set(this, context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue()); + publisher = (KafkaPublisher) TestUtils.getUnsafe().allocateInstance(KafkaPublisher.class); + publisher.setAckWaitTime(15000); + producer = mock(Producer.class); + this.instrumentProducer(producer, false); + Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer"); + kf.setAccessible(true); + kf.set(publisher, producer); + + Field componentLogF = KafkaPublisher.class.getDeclaredField("componentLog"); + componentLogF.setAccessible(true); + componentLogF.set(publisher, mock(ComponentLog.class)); + + Field ackCheckSizeField = KafkaPublisher.class.getDeclaredField("ackCheckSize"); + ackCheckSizeField.setAccessible(true); + ackCheckSizeField.set(publisher, this.ackCheckSize); + } catch (Exception e) { + e.printStackTrace(); + throw new IllegalStateException(e); + } + return publisher; + } + + @SuppressWarnings("unchecked") + private void instrumentProducer(Producer producer, boolean failRandomly) { + + when(producer.send(Mockito.any(ProducerRecord.class))).then(new Answer>() { + @Override + public Future answer(InvocationOnMock invocation) throws Throwable { + ProducerRecord record = (ProducerRecord) invocation.getArguments()[0]; + String value = new String(record.value(), StandardCharsets.UTF_8); + if ("fail".equals(value) && !StubPublishKafka.this.failed) { + StubPublishKafka.this.failed = true; + throw new RuntimeException("intentional"); + } + Future future = executor.submit(new Callable() { + @Override + public RecordMetadata call() throws Exception { + if ("futurefail".equals(value) && !StubPublishKafka.this.failed) { + StubPublishKafka.this.failed = true; + throw new TopicAuthorizationException("Unauthorized"); + } else { + TopicPartition partition = new TopicPartition("foo", 0); + RecordMetadata meta = new RecordMetadata(partition, 0, 0); + return meta; + } + } + }); + return future; + } + }); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java similarity index 99% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java index b056a0815d..819e3b73ac 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java @@ -32,7 +32,6 @@ class TestUtils { field.set(instance, newValue); } - static Unsafe getUnsafe() { try { Field f = Unsafe.class.getDeclaredField("theUnsafe"); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/log4j.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/log4j.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/log4j.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/server.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/server.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/server.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/server.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/zookeeper.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/zookeeper.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/resources/zookeeper.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/resources/zookeeper.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml similarity index 93% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/pom.xml rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml index 1b98f6a80d..92a6b291c9 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/pom.xml @@ -19,7 +19,7 @@ nifi-kafka-bundle 1.0.0-SNAPSHOT - nifi-kafka-nar + nifi-kafka-0-8-nar nar NiFi NAR for interacting with Apache Kafka @@ -29,7 +29,7 @@ org.apache.nifi - nifi-kafka-processors + nifi-kafka-0-8-processors diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/LICENSE similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/src/main/resources/META-INF/LICENSE rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/LICENSE diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-nar/src/main/resources/META-INF/NOTICE rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-nar/src/main/resources/META-INF/NOTICE diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml similarity index 93% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/pom.xml rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml index 98da7b2df6..ea498e68ff 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/pom.xml @@ -19,7 +19,7 @@ 1.0.0-SNAPSHOT 4.0.0 - nifi-kafka-processors + nifi-kafka-0-8-processors jar @@ -35,10 +35,10 @@ nifi-utils - org.apache.kafka - kafka-clients - 0.8.2.2 - + org.apache.kafka + kafka-clients + 0.8.2.2 +
org.apache.kafka kafka_2.10 diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/AbstractKafkaProcessor.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/Partitioners.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/Partitioners.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/Partitioners.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/Partitioners.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PublishingContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PublishingContext.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PublishingContext.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PublishingContext.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/GetKafkaIntegrationTests.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/GetKafkaIntegrationTests.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/GetKafkaIntegrationTests.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/GetKafkaIntegrationTests.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/PutKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/PutKafkaTest.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/PutKafkaTest.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/PutKafkaTest.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/log4j.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/log4j.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/log4j.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/server.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/server.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/server.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/server.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/zookeeper.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/zookeeper.properties similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/resources/zookeeper.properties rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/test/resources/zookeeper.properties diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml new file mode 100644 index 0000000000..3cc4d6689e --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/pom.xml @@ -0,0 +1,40 @@ + + + 4.0.0 + + org.apache.nifi + nifi-kafka-bundle + 1.0.0-SNAPSHOT + + nifi-kafka-0-9-nar + nar + NiFi NAR for interacting with Apache Kafka + + true + true + + + + org.apache.nifi + nifi-kafka-0-9-processors + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 0000000000..84b3bb95fb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,299 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +APACHE NIFI SUBCOMPONENTS: + +The Apache NiFi project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + + The binary distribution of this product bundles 'Scala Library' under a BSD + style license. + + Copyright (c) 2002-2015 EPFL + Copyright (c) 2011-2015 Typesafe, Inc. + + All rights reserved. + + Redistribution and use in source and binary forms, with or without modification, + are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this list of + conditions and the following disclaimer. + + Redistributions in binary form must reproduce the above copyright notice, this list of + conditions and the following disclaimer in the documentation and/or other materials + provided with the distribution. + + Neither the name of the EPFL nor the names of its contributors may be used to endorse + or promote products derived from this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS + OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR + CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER + IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT + OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + The binary distribution of this product bundles 'JLine' under a BSD + style license. + + Copyright (c) 2002-2006, Marc Prud'hommeaux + All rights reserved. + + Redistribution and use in source and binary forms, with or + without modification, are permitted provided that the following + conditions are met: + + Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with + the distribution. + + Neither the name of JLine nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, + BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO + EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE + FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, + OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED + AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING + IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + OF THE POSSIBILITY OF SUCH DAMAGE. + + The binary distribution of this product bundles 'JOpt Simple' under an MIT + style license. + + Copyright (c) 2009 Paul R. Holser, Jr. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..77ec4e965b --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,72 @@ +nifi-kafka-0-9-nar +Copyright 2014-2016 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2014 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + (ASLv2) Apache Kafka + The following NOTICE information applies: + Apache Kafka + Copyright 2012 The Apache Software Foundation. + + (ASLv2) Yammer Metrics + The following NOTICE information applies: + Metrics + Copyright 2010-2012 Coda Hale and Yammer, Inc. + + This product includes software developed by Coda Hale and Yammer, Inc. + + This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released + with the following comments: + + Written by Doug Lea with assistance from members of JCP JSR-166 + Expert Group and released to the public domain, as explained at + http://creativecommons.org/publicdomain/zero/1.0/ + + (ASLv2) Snappy Java + The following NOTICE information applies: + This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + + This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + + This library containd statically linked libstdc++. This inclusion is allowed by + "GCC RUntime Library Exception" + http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + + (ASLv2) Apache ZooKeeper + The following NOTICE information applies: + Apache ZooKeeper + Copyright 2009-2012 The Apache Software Foundation + +************************ +Common Development and Distribution License 1.1 +************************ + +The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail) + +************************ +Common Development and Distribution License 1.0 +************************ + +The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details. + + (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml similarity index 93% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml index 3ad8e37e0f..6216846700 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/pom.xml @@ -19,7 +19,7 @@ 1.0.0-SNAPSHOT 4.0.0 - nifi-kafka-pubsub-processors + nifi-kafka-0-9-processors jar @@ -39,14 +39,15 @@ nifi-ssl-context-service-api - org.apache.kafka - kafka-clients - 0.9.0.1 - + org.apache.kafka + kafka-clients + 0.9.0.1 + org.apache.kafka kafka_2.10 0.9.0.1 + test diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java new file mode 100644 index 0000000000..e5255f5764 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java @@ -0,0 +1,408 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.xml.bind.DatatypeConverter; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURITY_PROTOCOL; + +@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.9 Consumer API. " + + " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring" + + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can. In the mean time" + + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.") +@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.9.x"}) +@WritesAttributes({ + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY_HEX, description = "The hex encoded key of message if present and if single message"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic the message or message bundle is from") +}) +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.", + description = "These properties will be added on the Kafka configuration after loading any provided configuration properties." + + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged." + + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ") +public class ConsumeKafka extends AbstractProcessor { + + private static final long TWO_MB = 2L * 1024L * 1024L; + + static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset"); + + static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset"); + + static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group"); + + static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name(s)") + .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma seperated.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder() + .name(ConsumerConfig.GROUP_ID_CONFIG) + .displayName("Group ID") + .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder() + .name(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + .displayName("Offset Reset") + .description("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any " + + "more on the server (e.g. because that data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") + .required(true) + .allowableValues(OFFSET_EARLIEST, OFFSET_LATEST, OFFSET_NONE) + .defaultValue(OFFSET_LATEST.getValue()) + .build(); + + static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder() + .name("message-demarcator") + .displayName("Message Demarcator") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .description("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains " + + "all Kafka messages in a single batch for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use " + + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received " + + "will result in a single FlowFile which " + + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS") + .build(); + static final PropertyDescriptor MAX_POLL_RECORDS = new PropertyDescriptor.Builder() + .name("max.poll.records") + .displayName("Max Poll Records") + .description("Specifies the maximum number of records Kafka should return in a single poll.") + .required(false) + .defaultValue("10000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .build(); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles received from Kafka. Depending on demarcation strategy it is a flow file per message or a bundle of messages grouped by topic and partition.") + .build(); + + static final List DESCRIPTORS; + static final Set RELATIONSHIPS; + + private volatile byte[] demarcatorBytes = null; + private volatile ConsumerPool consumerPool = null; + + static { + List descriptors = new ArrayList<>(); + descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + descriptors.add(TOPICS); + descriptors.add(GROUP_ID); + descriptors.add(AUTO_OFFSET_RESET); + descriptors.add(MESSAGE_DEMARCATOR); + descriptors.add(MAX_POLL_RECORDS); + DESCRIPTORS = Collections.unmodifiableList(descriptors); + RELATIONSHIPS = Collections.singleton(REL_SUCCESS); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @OnScheduled + public void prepareProcessing(final ProcessContext context) { + this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() + ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) + : null; + } + + @OnStopped + public void close() { + demarcatorBytes = null; + final ConsumerPool pool = consumerPool; + consumerPool = null; + if (pool != null) { + pool.close(); + } + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.") + .name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ConsumerConfig.class)).dynamic(true) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + return KafkaProcessorUtils.validateCommonProperties(validationContext); + } + + private synchronized ConsumerPool getConsumerPool(final ProcessContext context) { + ConsumerPool pool = consumerPool; + if (pool != null) { + return pool; + } + + final Map props = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props); + final String topicListing = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue(); + final List topics = new ArrayList<>(); + for (final String topic : topicListing.split(",", 100)) { + final String trimmedName = topic.trim(); + if (!trimmedName.isEmpty()) { + topics.add(trimmedName); + } + } + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + return consumerPool = createConsumerPool(context.getMaxConcurrentTasks(), topics, props, getLogger()); + } + + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return new ConsumerPool(maxLeases, topics, props, log); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final long startTimeNanos = System.nanoTime(); + final ConsumerPool pool = getConsumerPool(context); + if (pool == null) { + context.yield(); + return; + } + final Map>> partitionRecordMap = new HashMap<>(); + + try (final ConsumerLease lease = pool.obtainConsumer()) { + try { + if (lease == null) { + context.yield(); + return; + } + + final boolean foundData = gatherDataFromKafka(lease, partitionRecordMap, context); + if (!foundData) { + session.rollback(); + return; + } + + writeSessionData(context, session, partitionRecordMap, startTimeNanos); + //At-least once commit handling (if order is reversed it is at-most once) + session.commit(); + commitOffsets(lease, partitionRecordMap); + } catch (final KafkaException ke) { + lease.poison(); + getLogger().error("Problem while accessing kafka consumer " + ke, ke); + context.yield(); + session.rollback(); + } + } + } + + private void commitOffsets(final ConsumerLease lease, final Map>> partitionRecordMap) { + final Map partOffsetMap = new HashMap<>(); + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .forEach((entry) -> { + long maxOffset = entry.getValue().stream() + .mapToLong(record -> record.offset()) + .max() + .getAsLong(); + partOffsetMap.put(entry.getKey(), new OffsetAndMetadata(maxOffset + 1L)); + }); + lease.commitOffsets(partOffsetMap); + } + + private void writeSessionData( + final ProcessContext context, final ProcessSession session, + final Map>> partitionRecordMap, + final long startTimeNanos) { + if (demarcatorBytes != null) { + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .forEach(entry -> { + writeData(context, session, entry.getValue(), startTimeNanos); + }); + } else { + partitionRecordMap.entrySet().stream() + .filter(entry -> !entry.getValue().isEmpty()) + .flatMap(entry -> entry.getValue().stream()) + .forEach(record -> { + writeData(context, session, Collections.singletonList(record), startTimeNanos); + }); + } + } + + private void writeData(final ProcessContext context, final ProcessSession session, final List> records, final long startTimeNanos) { + final ConsumerRecord firstRecord = records.get(0); + final String offset = String.valueOf(firstRecord.offset()); + final String keyHex = (firstRecord.key() != null) ? DatatypeConverter.printHexBinary(firstRecord.key()) : null; + final String topic = firstRecord.topic(); + final String partition = String.valueOf(firstRecord.partition()); + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, out -> { + boolean useDemarcator = false; + for (final ConsumerRecord record : records) { + if (useDemarcator) { + out.write(demarcatorBytes); + } + out.write(record.value()); + useDemarcator = true; + } + }); + final Map kafkaAttrs = new HashMap<>(); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, offset); + if (keyHex != null && records.size() == 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY_HEX, keyHex); + } + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, partition); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, topic); + if (records.size() > 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(records.size())); + } + flowFile = session.putAllAttributes(flowFile, kafkaAttrs); + final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); + final String transitUri = KafkaProcessorUtils.buildTransitURI( + context.getProperty(SECURITY_PROTOCOL).getValue(), + context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(), + topic); + session.getProvenanceReporter().receive(flowFile, transitUri, executionDurationMillis); + this.getLogger().debug("Created {} containing {} messages from Kafka topic {}, partition {}, starting offset {} in {} millis", + new Object[]{flowFile, records.size(), topic, partition, offset, executionDurationMillis}); + session.transfer(flowFile, REL_SUCCESS); + } + + /** + * Populates the given partitionRecordMap with new records until we poll + * that returns no records or until we have enough data. It is important to + * ensure we keep items grouped by their topic and partition so that when we + * bundle them we bundle them intelligently and so that we can set offsets + * properly even across multiple poll calls. + */ + private boolean gatherDataFromKafka(final ConsumerLease lease, final Map>> partitionRecordMap, ProcessContext context) { + final long startNanos = System.nanoTime(); + boolean foundData = false; + ConsumerRecords records; + final int maxRecords = context.getProperty(MAX_POLL_RECORDS).asInteger(); + + do { + records = lease.poll(); + + for (final TopicPartition partition : records.partitions()) { + List> currList = partitionRecordMap.get(partition); + if (currList == null) { + currList = new ArrayList<>(); + partitionRecordMap.put(partition, currList); + } + currList.addAll(records.records(partition)); + if (currList.size() > 0) { + foundData = true; + } + } + //If we received data and we still want to get more + } while (!records.isEmpty() && !checkIfGatheredEnoughData(partitionRecordMap, maxRecords, startNanos)); + return foundData; + } + + /** + * Determines if we have enough data as-is and should move on. + * + * @return true if we've been gathering for more than 500 ms or if we're + * demarcating and have more than 50 flowfiles worth or if we're per message + * and have more than 2000 flowfiles or if totalMessageSize is greater than + * two megabytes; false otherwise + * + * Implementation note: 500 millis and 5 MB are magic numbers. These may + * need to be tuned. They get at how often offsets will get committed to + * kafka relative to how many records will get buffered into memory in a + * poll call before writing to repos. + */ + private boolean checkIfGatheredEnoughData(final Map>> partitionRecordMap, final long maxRecords, final long startTimeNanos) { + + final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); + + if (durationMillis > 500) { + return true; + } + + int topicPartitionsFilled = 0; + int totalRecords = 0; + long totalRecordSize = 0; + + for (final List> recordList : partitionRecordMap.values()) { + if (!recordList.isEmpty()) { + topicPartitionsFilled++; + } + totalRecords += recordList.size(); + for (final ConsumerRecord rec : recordList) { + totalRecordSize += rec.value().length; + } + } + + if (demarcatorBytes != null && demarcatorBytes.length > 0) { + return topicPartitionsFilled > 50; + } else if (totalRecordSize > TWO_MB) { + return true; + } else { + return totalRecords > maxRecords; + } + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java new file mode 100644 index 0000000000..b954eba344 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.io.Closeable; +import java.util.Map; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +/** + * This class represents a lease to access a Kafka Consumer object. The lease is + * intended to be obtained from a ConsumerPool. The lease is closeable to allow + * for the clean model of a try w/resources whereby non-exceptional cases mean + * the lease will be returned to the pool for future use by others. A given + * lease may only belong to a single thread a time. + */ +public interface ConsumerLease extends Closeable { + + /** + * Executes a poll on the underlying Kafka Consumer. + * + * @return ConsumerRecords retrieved in the poll. + * @throws KafkaException if issue occurs talking to underlying resource. + */ + ConsumerRecords poll() throws KafkaException; + + /** + * Notifies Kafka to commit the offsets for the specified topic/partition + * pairs to the specified offsets w/the given metadata. This can offer + * higher performance than the other commitOffsets call as it allows the + * kafka client to collect more data from Kafka before committing the + * offsets. + * + * @param offsets offsets + * @throws KafkaException if issue occurs talking to underlying resource. + */ + void commitOffsets(Map offsets) throws KafkaException; + + /** + * Notifies that this lease is poisoned and should not be reused. + */ + void poison(); + + /** + * Notifies that this lease is to be returned. The pool may optionally reuse + * this lease with another client. No further references by the caller + * should occur after calling close. + */ + @Override + void close(); + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java new file mode 100644 index 0000000000..3f20b8f4cb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.nifi.logging.ComponentLog; + +import java.io.Closeable; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +/** + * A pool of Kafka Consumers for a given topic. Consumers can be obtained by + * calling 'obtainConsumer'. Once closed the pool is ready to be immediately + * used again. + */ +public class ConsumerPool implements Closeable { + + private final AtomicInteger activeLeaseCount = new AtomicInteger(0); + private final int maxLeases; + private final Queue consumerLeases; + private final List topics; + private final Map kafkaProperties; + private final ComponentLog logger; + + private final AtomicLong consumerCreatedCountRef = new AtomicLong(); + private final AtomicLong consumerClosedCountRef = new AtomicLong(); + private final AtomicLong leasesObtainedCountRef = new AtomicLong(); + private final AtomicLong productivePollCountRef = new AtomicLong(); + private final AtomicLong unproductivePollCountRef = new AtomicLong(); + + /** + * Creates a pool of KafkaConsumer objects that will grow up to the maximum + * indicated leases. Consumers are lazily initialized. + * + * @param maxLeases maximum number of active leases in the pool + * @param topics the topics to consume from + * @param kafkaProperties the properties for each consumer + * @param logger the logger to report any errors/warnings + */ + public ConsumerPool(final int maxLeases, final List topics, final Map kafkaProperties, final ComponentLog logger) { + this.maxLeases = maxLeases; + if (maxLeases <= 0) { + throw new IllegalArgumentException("Max leases value must be greather than zero."); + } + this.logger = logger; + if (topics == null || topics.isEmpty()) { + throw new IllegalArgumentException("Must have a list of one or more topics"); + } + this.topics = topics; + this.kafkaProperties = new HashMap<>(kafkaProperties); + this.consumerLeases = new ArrayDeque<>(); + } + + /** + * Obtains a consumer from the pool if one is available + * + * @return consumer from the pool + * @throws IllegalArgumentException if pool already contains + */ + public ConsumerLease obtainConsumer() { + final ConsumerLease lease; + final int activeLeases; + synchronized (this) { + lease = consumerLeases.poll(); + activeLeases = activeLeaseCount.get(); + } + if (lease == null && activeLeases >= maxLeases) { + logger.warn("No available consumers and cannot create any as max consumer leases limit reached - verify pool settings"); + return null; + } + leasesObtainedCountRef.incrementAndGet(); + return (lease == null) ? createConsumer() : lease; + } + + protected Consumer createKafkaConsumer() { + return new KafkaConsumer<>(kafkaProperties); + } + + private ConsumerLease createConsumer() { + final Consumer kafkaConsumer = createKafkaConsumer(); + consumerCreatedCountRef.incrementAndGet(); + try { + kafkaConsumer.subscribe(topics); + } catch (final KafkaException kex) { + try { + kafkaConsumer.close(); + consumerClosedCountRef.incrementAndGet(); + } catch (final Exception ex) { + consumerClosedCountRef.incrementAndGet(); + //ignore + } + throw kex; + } + + final ConsumerLease lease = new ConsumerLease() { + + private volatile boolean poisoned = false; + private volatile boolean closed = false; + + @Override + public ConsumerRecords poll() { + + if (poisoned) { + throw new KafkaException("The consumer is poisoned and should no longer be used"); + } + + try { + final ConsumerRecords records = kafkaConsumer.poll(50); + if (records.isEmpty()) { + unproductivePollCountRef.incrementAndGet(); + } else { + productivePollCountRef.incrementAndGet(); + } + return records; + } catch (final KafkaException kex) { + logger.warn("Unable to poll from Kafka consumer so will poison and close this " + kafkaConsumer, kex); + poison(); + close(); + throw kex; + } + } + + @Override + public void commitOffsets(final Map offsets) { + + if (poisoned) { + throw new KafkaException("The consumer is poisoned and should no longer be used"); + } + try { + kafkaConsumer.commitSync(offsets); + } catch (final KafkaException kex) { + logger.warn("Unable to commit kafka consumer offsets so will poison and close this " + kafkaConsumer, kex); + poison(); + close(); + throw kex; + } + } + + @Override + public void close() { + if (closed) { + return; + } + if (poisoned || activeLeaseCount.get() > maxLeases) { + closeConsumer(kafkaConsumer); + activeLeaseCount.decrementAndGet(); + closed = true; + } else { + final boolean added; + synchronized (ConsumerPool.this) { + added = consumerLeases.offer(this); + } + if (!added) { + closeConsumer(kafkaConsumer); + activeLeaseCount.decrementAndGet(); + } + } + } + + @Override + public void poison() { + poisoned = true; + } + }; + activeLeaseCount.incrementAndGet(); + return lease; + } + + /** + * Closes all consumers in the pool. Can be safely recalled. + */ + @Override + public void close() { + final List leases = new ArrayList<>(); + synchronized (this) { + ConsumerLease lease = null; + while ((lease = consumerLeases.poll()) != null) { + leases.add(lease); + } + } + for (final ConsumerLease lease : leases) { + lease.poison(); + lease.close(); + } + } + + private void closeConsumer(final Consumer consumer) { + try { + consumer.unsubscribe(); + } catch (Exception e) { + logger.warn("Failed while unsubscribing " + consumer, e); + } + + try { + consumer.close(); + consumerClosedCountRef.incrementAndGet(); + } catch (Exception e) { + consumerClosedCountRef.incrementAndGet(); + logger.warn("Failed while closing " + consumer, e); + } + } + + PoolStats getPoolStats() { + return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get(), productivePollCountRef.get(), unproductivePollCountRef.get()); + } + + static final class PoolStats { + + final long consumerCreatedCount; + final long consumerClosedCount; + final long leasesObtainedCount; + final long productivePollCount; + final long unproductivePollCount; + + PoolStats( + final long consumerCreatedCount, + final long consumerClosedCount, + final long leasesObtainedCount, + final long productivePollCount, + final long unproductivePollCount + ) { + this.consumerCreatedCount = consumerCreatedCount; + this.consumerClosedCount = consumerClosedCount; + this.leasesObtainedCount = leasesObtainedCount; + this.productivePollCount = productivePollCount; + this.unproductivePollCount = unproductivePollCount; + } + + @Override + public String toString() { + return "Created Consumers [" + consumerCreatedCount + "]\n" + + "Closed Consumers [" + consumerClosedCount + "]\n" + + "Leases Obtained [" + leasesObtainedCount + "]\n" + + "Productive Polls [" + productivePollCount + "]\n" + + "Unproductive Polls [" + unproductivePollCount + "]\n"; + } + + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java new file mode 100644 index 0000000000..fd747fca02 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; +import org.apache.kafka.clients.CommonClientConfigs; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class KafkaProcessorUtils { + + final Logger logger = LoggerFactory.getLogger(this.getClass()); + + private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}"; + + private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*"; + + static final Pattern HEX_KEY_PATTERN = Pattern.compile("(?:[0123456789abcdefABCDEF]{2})+"); + + static final String KAFKA_KEY_HEX = "kafka.key.hex"; + static final String KAFKA_TOPIC = "kafka.topic"; + static final String KAFKA_PARTITION = "kafka.partition"; + static final String KAFKA_OFFSET = "kafka.offset"; + static final String KAFKA_COUNT = "kafka.count"; + static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT"); + static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL"); + static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT"); + static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL"); + + static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder() + .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) + .displayName("Kafka Brokers") + .description("A comma-separated list of known Kafka Brokers in the format :") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) + .expressionLanguageSupported(true) + .defaultValue("localhost:9092") + .build(); + static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder() + .name("security.protocol") + .displayName("Security Protocol") + .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") + .required(true) + .expressionLanguageSupported(false) + .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL) + .defaultValue(SEC_PLAINTEXT.getValue()) + .build(); + static final PropertyDescriptor KERBEROS_PRINCIPLE = new PropertyDescriptor.Builder() + .name("sasl.kerberos.service.name") + .displayName("Kerberos Service Name") + .description("The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config. " + + "Corresponds to Kafka's 'security.protocol' property." + + "It is ignored unless one of the SASL options of the are selected.") + .required(false) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("ssl.context.service") + .displayName("SSL Context Service") + .description("Specifies the SSL Context Service to use for communicating with Kafka.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + + static List getCommonPropertyDescriptors() { + return Arrays.asList( + BOOTSTRAP_SERVERS, + SECURITY_PROTOCOL, + KERBEROS_PRINCIPLE, + SSL_CONTEXT_SERVICE + ); + } + + static Collection validateCommonProperties(final ValidationContext validationContext) { + List results = new ArrayList<>(); + + String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue(); + + /* + * validates that if one of SASL (Kerberos) option is selected for + * security protocol, then Kerberos principal is provided as well + */ + if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) { + String kerberosPrincipal = validationContext.getProperty(KERBEROS_PRINCIPLE).getValue(); + if (kerberosPrincipal == null || kerberosPrincipal.trim().length() == 0) { + results.add(new ValidationResult.Builder().subject(KERBEROS_PRINCIPLE.getDisplayName()).valid(false) + .explanation("The <" + KERBEROS_PRINCIPLE.getDisplayName() + "> property must be set when <" + + SECURITY_PROTOCOL.getDisplayName() + "> is configured as '" + + SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.") + .build()); + } + } + + //If SSL or SASL_SSL then CS must be set. + final boolean sslProtocol = SEC_SSL.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol); + final boolean csSet = validationContext.getProperty(SSL_CONTEXT_SERVICE).isSet(); + if (csSet && !sslProtocol) { + results.add(new ValidationResult.Builder().subject(SECURITY_PROTOCOL.getDisplayName()).valid(false) + .explanation("If you set the SSL Controller Service you should also choose an SSL based security protocol.").build()); + } + if (!csSet && sslProtocol) { + results.add(new ValidationResult.Builder().subject(SSL_CONTEXT_SERVICE.getDisplayName()).valid(false) + .explanation("If you set to an SSL based protocol you need to set the SSL Controller Service").build()); + } + + final String enableAutoCommit = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).build()).getValue(); + if (enableAutoCommit != null && !enableAutoCommit.toLowerCase().equals("false")) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + .explanation("Enable auto commit must be false. It is managed by the processor.").build()); + } + + final String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()).getValue(); + if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) { + results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG) + .explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build()); + } + + final String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()).getValue(); + if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) { + results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG) + .explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build()); + } + + final String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()).getValue(); + if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG) + .explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build()); + } + + final String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()).getValue(); + if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) { + results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG) + .explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build()); + } + + return results; + } + + static final class KafkaConfigValidator implements Validator { + + final Class classType; + + public KafkaConfigValidator(final Class classType) { + this.classType = classType; + } + + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + final boolean knownValue = KafkaProcessorUtils.isStaticStringFieldNamePresent(subject, classType, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class); + return new ValidationResult.Builder().subject(subject).explanation("Must be a known configuration parameter for this kafka client").valid(knownValue).build(); + } + }; + + /** + * Builds transit URI for provenance event. The transit URI will be in the + * form of <security.protocol>://<bootstrap.servers>/topic + */ + static String buildTransitURI(String securityProtocol, String brokers, String topic) { + StringBuilder builder = new StringBuilder(); + builder.append(securityProtocol); + builder.append("://"); + builder.append(brokers); + builder.append("/"); + builder.append(topic); + return builder.toString(); + } + + static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map mapToPopulate) { + for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) { + if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) { + // Translate SSLContext Service configuration into Kafka properties + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + if (sslContextService != null && sslContextService.isKeyStoreConfigured()) { + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, sslContextService.getKeyStoreFile()); + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sslContextService.getKeyStorePassword()); + final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword(); + mapToPopulate.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPass); + mapToPopulate.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, sslContextService.getKeyStoreType()); + } + + if (sslContextService != null && sslContextService.isTrustStoreConfigured()) { + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslContextService.getTrustStoreFile()); + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, sslContextService.getTrustStorePassword()); + mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType()); + } + } + + String pName = propertyDescriptor.getName(); + String pValue = propertyDescriptor.isExpressionLanguageSupported() + ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue() + : context.getProperty(propertyDescriptor).getValue(); + if (pValue != null) { + if (pName.endsWith(".ms")) { // kafka standard time notation + pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS)); + } + if (isStaticStringFieldNamePresent(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) { + mapToPopulate.put(pName, pValue); + } + } + } + } + + private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) { + return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name); + } + + private static Set getPublicStaticStringFieldValues(final Class... classes) { + final Set strings = new HashSet<>(); + for (final Class classType : classes) { + for (final Field field : classType.getDeclaredFields()) { + if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) { + try { + strings.add(String.valueOf(field.get(null))); + } catch (IllegalArgumentException | IllegalAccessException ex) { + //ignore + } + } + } + } + return strings; + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java new file mode 100644 index 0000000000..31a084f133 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.io.Closeable; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.stream.io.util.StreamDemarcator; + +/** + * Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor + * with sending contents of the {@link FlowFile}s to Kafka. + */ +class KafkaPublisher implements Closeable { + + private final Producer kafkaProducer; + + private volatile long ackWaitTime = 30000; + + private final ComponentLog componentLog; + + private final int ackCheckSize; + + KafkaPublisher(Properties kafkaProperties, ComponentLog componentLog) { + this(kafkaProperties, 100, componentLog); + } + + /** + * Creates an instance of this class as well as the instance of the + * corresponding Kafka {@link KafkaProducer} using provided Kafka + * configuration properties. + * + * @param kafkaProperties instance of {@link Properties} used to bootstrap + * {@link KafkaProducer} + */ + KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) { + this.kafkaProducer = new KafkaProducer<>(kafkaProperties); + this.ackCheckSize = ackCheckSize; + this.componentLog = componentLog; + } + + /** + * Publishes messages to Kafka topic. It uses {@link StreamDemarcator} to + * determine how many messages to Kafka will be sent from a provided + * {@link InputStream} (see {@link PublishingContext#getContentStream()}). + * It supports two publishing modes: + *
    + *
  • Sending all messages constructed from + * {@link StreamDemarcator#nextToken()} operation.
  • + *
  • Sending only unacknowledged messages constructed from + * {@link StreamDemarcator#nextToken()} operation.
  • + *
+ * The unacknowledged messages are determined from the value of + * {@link PublishingContext#getLastAckedMessageIndex()}. + *
+ * This method assumes content stream affinity where it is expected that the + * content stream that represents the same Kafka message(s) will remain the + * same across possible retries. This is required specifically for cases + * where delimiter is used and a single content stream may represent + * multiple Kafka messages. The + * {@link PublishingContext#getLastAckedMessageIndex()} will provide the + * index of the last ACKed message, so upon retry only messages with the + * higher index are sent. + * + * @param publishingContext instance of {@link PublishingContext} which hold + * context information about the message(s) to be sent. + * @return The index of the last successful offset. + */ + KafkaPublisherResult publish(PublishingContext publishingContext) { + StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(), + publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize()); + + int prevLastAckedMessageIndex = publishingContext.getLastAckedMessageIndex(); + List> resultFutures = new ArrayList<>(); + + byte[] messageBytes; + int tokenCounter = 0; + boolean continueSending = true; + KafkaPublisherResult result = null; + for (; continueSending && (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) { + if (prevLastAckedMessageIndex < tokenCounter) { + ProducerRecord message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes); + resultFutures.add(this.kafkaProducer.send(message)); + + if (tokenCounter % this.ackCheckSize == 0) { + int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex); + resultFutures.clear(); + if (lastAckedMessageIndex % this.ackCheckSize != 0) { + continueSending = false; + result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex); + } + prevLastAckedMessageIndex = lastAckedMessageIndex; + } + } + } + + if (result == null) { + int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex); + resultFutures.clear(); + result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex); + } + return result; + } + + /** + * Sets the time this publisher will wait for the {@link Future#get()} + * operation (the Future returned by + * {@link KafkaProducer#send(ProducerRecord)}) to complete before timing + * out. + * + * This value will also be used as a timeout when closing the underlying + * {@link KafkaProducer}. See {@link #close()}. + */ + void setAckWaitTime(long ackWaitTime) { + this.ackWaitTime = ackWaitTime; + } + + /** + * This operation will process ACKs from Kafka in the order in which + * {@link KafkaProducer#send(ProducerRecord)} invocation were made returning + * the index of the last ACKed message. Within this operation processing ACK + * simply means successful invocation of 'get()' operation on the + * {@link Future} returned by {@link KafkaProducer#send(ProducerRecord)} + * operation. Upon encountering any type of error while interrogating such + * {@link Future} the ACK loop will end. Messages that were not ACKed would + * be considered non-delivered and therefore could be resent at the later + * time. + * + * @param sendFutures list of {@link Future}s representing results of + * publishing to Kafka + * + * @param lastAckMessageIndex the index of the last ACKed message. It is + * important to provide the last ACKed message especially while re-trying so + * the proper index is maintained. + */ + private int processAcks(List> sendFutures, int lastAckMessageIndex) { + boolean exceptionThrown = false; + for (int segmentCounter = 0; segmentCounter < sendFutures.size() && !exceptionThrown; segmentCounter++) { + Future future = sendFutures.get(segmentCounter); + try { + future.get(this.ackWaitTime, TimeUnit.MILLISECONDS); + lastAckMessageIndex++; + } catch (InterruptedException e) { + exceptionThrown = true; + Thread.currentThread().interrupt(); + this.warnOrError("Interrupted while waiting for acks from Kafka", null); + } catch (ExecutionException e) { + exceptionThrown = true; + this.warnOrError("Failed while waiting for acks from Kafka", e); + } catch (TimeoutException e) { + exceptionThrown = true; + this.warnOrError("Timed out while waiting for acks from Kafka", null); + } + } + + return lastAckMessageIndex; + } + + /** + * Will close the underlying {@link KafkaProducer} waiting if necessary for + * the same duration as supplied {@link #setAckWaitTime(long)} + */ + @Override + public void close() { + this.kafkaProducer.close(this.ackWaitTime, TimeUnit.MILLISECONDS); + } + + /** + * + */ + private void warnOrError(String message, Exception e) { + if (e == null) { + this.componentLog.warn(message); + } else { + this.componentLog.error(message, e); + } + } + + /** + * Encapsulates the result received from publishing messages to Kafka + */ + static class KafkaPublisherResult { + + private final int messagesSent; + private final int lastMessageAcked; + + KafkaPublisherResult(int messagesSent, int lastMessageAcked) { + this.messagesSent = messagesSent; + this.lastMessageAcked = lastMessageAcked; + } + + public int getMessagesSent() { + return this.messagesSent; + } + + public int getLastMessageAcked() { + return this.lastMessageAcked; + } + + public boolean isAllAcked() { + return this.lastMessageAcked > -1 && this.messagesSent - 1 == this.lastMessageAcked; + } + + @Override + public String toString() { + return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked; + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java new file mode 100644 index 0000000000..64ab4ce9c8 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.util.Map; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; + +/** + * Collection of implementation of common Kafka {@link Partitioner}s. + */ +final public class Partitioners { + + private Partitioners() { + } + + /** + * {@link Partitioner} that implements 'round-robin' mechanism which evenly + * distributes load between all available partitions. + */ + public static class RoundRobinPartitioner implements Partitioner { + + private volatile int index; + + @Override + public void configure(Map configs) { + // noop + } + + @Override + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) { + return this.next(cluster.availablePartitionsForTopic(topic).size()); + } + + @Override + public void close() { + // noop + } + + private synchronized int next(int numberOfPartitions) { + if (this.index >= numberOfPartitions) { + this.index = 0; + } + return index++; + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java similarity index 57% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java index 6703c04a5b..4745984563 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java @@ -16,10 +16,12 @@ */ package org.apache.nifi.processors.kafka.pubsub; +import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -28,35 +30,48 @@ import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; - +import javax.xml.bind.DatatypeConverter; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.InputRequirement; -import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractSessionFactoryProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult; -import org.apache.nifi.processors.kafka.pubsub.Partitioners.RoundRobinPartitioner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -@InputRequirement(Requirement.INPUT_REQUIRED) -@Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" }) -@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka. The messages to send may be individual FlowFiles or may be delimited, using a " - + "user-specified delimiter, such as a new-line.") +@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.9.x"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.9 producer. " + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring" + + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can. In the mean time" + + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.", - description = "These properties will be added on the Kafka configuration after loading any provided configuration properties." + description = "These properties will be added on the Kafka configuration after loading any provided configuration properties." + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged." - + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.") -public class PublishKafka extends AbstractKafkaProcessor { + + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ") +public class PublishKafka extends AbstractSessionFactoryProcessor { + + private final Logger logger = LoggerFactory.getLogger(this.getClass()); protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id"; @@ -72,22 +87,31 @@ public class PublishKafka extends AbstractKafkaProcessor { static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery", "FlowFile will be routed to failure unless the message is replicated to the appropriate " - + "number of Kafka Nodes according to the Topic configuration"); + + "number of Kafka Nodes according to the Topic configuration"); static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery", "FlowFile will be routed to success if the message is received by a single Kafka node, " - + "whether or not it is replicated. This is faster than " - + "but can result in data loss if a Kafka node crashes"); + + "whether or not it is replicated. This is faster than " + + "but can result in data loss if a Kafka node crashes"); static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort", "FlowFile will be routed to success after successfully writing the content to a Kafka node, " - + "without waiting for a response. This provides the best performance but may result in data loss."); + + "without waiting for a response. This provides the best performance but may result in data loss."); - static final AllowableValue ROUND_ROBIN_PARTITIONING = new AllowableValue(RoundRobinPartitioner.class.getName(), - RoundRobinPartitioner.class.getSimpleName(), + static final AllowableValue ROUND_ROBIN_PARTITIONING = new AllowableValue(Partitioners.RoundRobinPartitioner.class.getName(), + Partitioners.RoundRobinPartitioner.class.getSimpleName(), "Messages will be assigned partitions in a round-robin fashion, sending the first message to Partition 1, " - + "the next Partition to Partition 2, and so on, wrapping as necessary."); + + "the next Partition to Partition 2, and so on, wrapping as necessary."); static final AllowableValue RANDOM_PARTITIONING = new AllowableValue("org.apache.kafka.clients.producer.internals.DefaultPartitioner", "DefaultPartitioner", "Messages will be assigned to random partitions."); + static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() + .name("topic") + .displayName("Topic Name") + .description("The name of the Kafka Topic to publish to.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder() .name(ProducerConfig.ACKS_CONFIG) .displayName("Delivery Guarantee") @@ -97,30 +121,42 @@ public class PublishKafka extends AbstractKafkaProcessor { .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED) .defaultValue(DELIVERY_BEST_EFFORT.getValue()) .build(); + static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder() .name(ProducerConfig.MAX_BLOCK_MS_CONFIG) .displayName("Meta Data Wait Time") .description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the " - + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property") + + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property") .required(true) .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) .expressionLanguageSupported(true) .defaultValue("30 sec") .build(); + static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() .name("kafka-key") .displayName("Kafka Key") - .description("The Key to use for the Message") + .description("The Key to use for the Message. It will be serialized as UTF-8 bytes. " + + "If not specified then the flow file attribute kafka.key.hex is used if present " + + "and we're not demarcating. In that case the hex string is coverted to its byte" + + "form and written as a byte[] key.") .required(false) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(true) .build(); - static final PropertyDescriptor MESSAGE_DEMARCATOR = MESSAGE_DEMARCATOR_BUILDER - .description("Specifies the string (interpreted as UTF-8) to use for demarcating apart multiple messages within " + + static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder() + .name("message-demarcator") + .displayName("Message Demarcator") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .description("Specifies the string (interpreted as UTF-8) to use for demarcating multiple messages within " + "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the " - + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. " - + "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on your OS.") + + "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. " + + "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on your OS.") .build(); + static final PropertyDescriptor PARTITION_CLASS = new PropertyDescriptor.Builder() .name(ProducerConfig.PARTITIONER_CLASS_CONFIG) .displayName("Partitioner class") @@ -129,6 +165,7 @@ public class PublishKafka extends AbstractKafkaProcessor { .defaultValue(RANDOM_PARTITIONING.getValue()) .required(false) .build(); + static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder() .name(ProducerConfig.COMPRESSION_TYPE_CONFIG) .displayName("Compression Type") @@ -139,6 +176,11 @@ public class PublishKafka extends AbstractKafkaProcessor { .defaultValue("none") .build(); + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles for which all content was sent to Kafka.") + .build(); + static final Relationship REL_FAILURE = new Relationship.Builder() .name("failure") .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship") @@ -150,13 +192,18 @@ public class PublishKafka extends AbstractKafkaProcessor { private volatile String brokers; + private final AtomicInteger taskCounter = new AtomicInteger(); + + private volatile boolean acceptTask = true; + /* * Will ensure that list of PropertyDescriptors is build only once, since * all other lifecycle methods are invoked multiple times. */ static { - List _descriptors = new ArrayList<>(); - _descriptors.addAll(SHARED_DESCRIPTORS); + final List _descriptors = new ArrayList<>(); + _descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + _descriptors.add(TOPIC); _descriptors.add(DELIVERY_GUARANTEE); _descriptors.add(KEY); _descriptors.add(MESSAGE_DEMARCATOR); @@ -166,39 +213,141 @@ public class PublishKafka extends AbstractKafkaProcessor { DESCRIPTORS = Collections.unmodifiableList(_descriptors); - Set _relationships = new HashSet<>(); - _relationships.addAll(SHARED_RELATIONSHIPS); + final Set _relationships = new HashSet<>(); + _relationships.add(REL_SUCCESS); _relationships.add(REL_FAILURE); RELATIONSHIPS = Collections.unmodifiableSet(_relationships); } - /** - * - */ @Override public Set getRelationships() { return RELATIONSHIPS; } - /** - * - */ @Override protected List getSupportedPropertyDescriptors() { return DESCRIPTORS; } + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.") + .name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + return KafkaProcessorUtils.validateCommonProperties(validationContext); + } + + volatile KafkaPublisher kafkaPublisher; + /** - * Will rendezvous with Kafka if {@link ProcessSession} contains {@link FlowFile} - * producing a result {@link FlowFile}. - *
- * The result {@link FlowFile} that is successful is then transfered to {@link #REL_SUCCESS} - *
- * The result {@link FlowFile} that is failed is then transfered to {@link #REL_FAILURE} + * This thread-safe operation will delegate to + * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first + * checking and creating (if necessary) Kafka resource which could be either + * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and + * destroy the underlying Kafka resource upon catching an {@link Exception} + * raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}. + * After Kafka resource is destroyed it will be re-created upon the next + * invocation of this operation essentially providing a self healing + * mechanism to deal with potentially corrupted resource. + *

+ * Keep in mind that upon catching an exception the state of this processor + * will be set to no longer accept any more tasks, until Kafka resource is + * reset. This means that in a multi-threaded situation currently executing + * tasks will be given a chance to complete while no new tasks will be + * accepted. * + * @param context context + * @param sessionFactory factory */ @Override - protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session){ + public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException { + if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted. + this.taskCounter.incrementAndGet(); + final ProcessSession session = sessionFactory.createSession(); + try { + /* + * We can't be doing double null check here since as a pattern + * it only works for lazy init but not reset, which is what we + * are doing here. In fact the first null check is dangerous + * since 'kafkaPublisher' can become null right after its null + * check passed causing subsequent NPE. + */ + synchronized (this) { + if (this.kafkaPublisher == null) { + this.kafkaPublisher = this.buildKafkaResource(context, session); + } + } + + /* + * The 'processed' boolean flag does not imply any failure or success. It simply states that: + * - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated + * - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile + */ + boolean processed = this.rendezvousWithKafka(context, session); + session.commit(); + if (!processed) { + context.yield(); + } + } catch (Throwable e) { + this.acceptTask = false; + session.rollback(true); + this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e}); + } finally { + synchronized (this) { + if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) { + this.close(); + this.acceptTask = true; + } + } + } + } else { + this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); + this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); + context.yield(); + } + } + + /** + * Will call {@link Closeable#close()} on the target resource after which + * the target resource will be set to null. Should only be called when there + * are no more threads being executed on this processor or when it has been + * verified that only a single thread remains. + * + * @see KafkaPublisher + * @see KafkaConsumer + */ + @OnStopped + public void close() { + try { + if (this.kafkaPublisher != null) { + try { + this.kafkaPublisher.close(); + } catch (Exception e) { + this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e); + } + } + } finally { + this.kafkaPublisher = null; + } + } + + /** + * Will rendezvous with Kafka if {@link ProcessSession} contains + * {@link FlowFile} producing a result {@link FlowFile}. + *
+ * The result {@link FlowFile} that is successful is then transfered to + * {@link #REL_SUCCESS} + *
+ * The result {@link FlowFile} that is failed is then transfered to + * {@link #REL_FAILURE} + * + */ + protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) { FlowFile flowFile = session.get(); if (flowFile != null) { long start = System.nanoTime(); @@ -207,9 +356,10 @@ public class PublishKafka extends AbstractKafkaProcessor { if (!this.isFailedFlowFile(flowFile)) { String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); - String transitUri = this.buildTransitURI(context.getProperty(SECURITY_PROTOCOL).getValue(), this.brokers, topic); + String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic); session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration); - this.getLogger().info("Successfully sent {} to Kafka as {} message(s) in {} millis", new Object[] { flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration }); + this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis", + new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration}); } else { relationship = REL_FAILURE; flowFile = session.penalize(flowFile); @@ -222,31 +372,33 @@ public class PublishKafka extends AbstractKafkaProcessor { /** * Builds and instance of {@link KafkaPublisher}. */ - @Override protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) { - Properties kafkaProperties = this.buildKafkaProperties(context); - kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); - kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); - this.brokers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); - KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, this.getLogger()); + final Map kafkaProps = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps); + kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + final Properties props = new Properties(); + props.putAll(kafkaProps); + KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger()); return publisher; } /** * Will rendezvous with {@link KafkaPublisher} after building - * {@link PublishingContext} and will produce the resulting {@link FlowFile}. - * The resulting FlowFile contains all required information to determine - * if message publishing originated from the provided FlowFile has actually - * succeeded fully, partially or failed completely (see - * {@link #isFailedFlowFile(FlowFile)}. + * {@link PublishingContext} and will produce the resulting + * {@link FlowFile}. The resulting FlowFile contains all required + * information to determine if message publishing originated from the + * provided FlowFile has actually succeeded fully, partially or failed + * completely (see {@link #isFailedFlowFile(FlowFile)}. */ private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) { - final AtomicReference publishResultRef = new AtomicReference<>(); + final AtomicReference publishResultRef = new AtomicReference<>(); session.read(flowFile, new InputStreamCallback() { @Override public void process(InputStream contentStream) throws IOException { PublishingContext publishingContext = PublishKafka.this.buildPublishingContext(flowFile, context, contentStream); - KafkaPublisherResult result = PublishKafka.this.kafkaResource.publish(publishingContext); + KafkaPublisher.KafkaPublisherResult result = PublishKafka.this.kafkaPublisher.publish(publishingContext); publishResultRef.set(result); } }); @@ -256,7 +408,7 @@ public class PublishKafka extends AbstractKafkaProcessor { : session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context)); if (!this.isFailedFlowFile(resultFile)) { - resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent())); + resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent())); } return resultFile; } @@ -281,7 +433,7 @@ public class PublishKafka extends AbstractKafkaProcessor { * only considered a failed FlowFile if it is being re-processed by the same * processor (determined via {@link #FAILED_PROC_ID_ATTR}, see * {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to - * another PublishKafka processor it is treated as a fresh FlowFile + * another PublishKafka0_10 processor it is treated as a fresh FlowFile * regardless if it has #FAILED* attributes set. */ private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) { @@ -301,6 +453,10 @@ public class PublishKafka extends AbstractKafkaProcessor { topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); keyBytes = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8); + String keyHex = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_KEY_HEX); + if (_key == null && keyHex != null && KafkaProcessorUtils.HEX_KEY_PATTERN.matcher(keyHex).matches()) { + keyBytes = DatatypeConverter.parseHexBinary(keyHex); + } delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR) .evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null; } @@ -346,7 +502,7 @@ public class PublishKafka extends AbstractKafkaProcessor { attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue()); attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue()); attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet() - ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null); + ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null); return attributes; } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java new file mode 100644 index 0000000000..bda29e6410 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.io.InputStream; +import java.nio.charset.StandardCharsets; + +/** + * Holder of context information used by {@link KafkaPublisher} required to + * publish messages to Kafka. + */ +class PublishingContext { + + private final InputStream contentStream; + + private final String topic; + + private final int lastAckedMessageIndex; + + /* + * We're using the default value from Kafka. We are using it to control the + * message size before it goes to to Kafka thus limiting possibility of a + * late failures in Kafka client. + */ + private int maxRequestSize = 1048576; // kafka default + + private boolean maxRequestSizeSet; + + private byte[] keyBytes; + + private byte[] delimiterBytes; + + PublishingContext(InputStream contentStream, String topic) { + this(contentStream, topic, -1); + } + + PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) { + this.validateInput(contentStream, topic, lastAckedMessageIndex); + this.contentStream = contentStream; + this.topic = topic; + this.lastAckedMessageIndex = lastAckedMessageIndex; + } + + @Override + public String toString() { + return "topic: '" + this.topic + "'; delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'"; + } + + int getLastAckedMessageIndex() { + return this.lastAckedMessageIndex; + } + + int getMaxRequestSize() { + return this.maxRequestSize; + } + + byte[] getKeyBytes() { + return this.keyBytes; + } + + byte[] getDelimiterBytes() { + return this.delimiterBytes; + } + + InputStream getContentStream() { + return this.contentStream; + } + + String getTopic() { + return this.topic; + } + + void setKeyBytes(byte[] keyBytes) { + if (this.keyBytes == null) { + if (keyBytes != null) { + this.assertBytesValid(keyBytes); + this.keyBytes = keyBytes; + } + } else { + throw new IllegalArgumentException("'keyBytes' can only be set once per instance"); + } + } + + void setDelimiterBytes(byte[] delimiterBytes) { + if (this.delimiterBytes == null) { + if (delimiterBytes != null) { + this.assertBytesValid(delimiterBytes); + this.delimiterBytes = delimiterBytes; + } + } else { + throw new IllegalArgumentException("'delimiterBytes' can only be set once per instance"); + } + } + + void setMaxRequestSize(int maxRequestSize) { + if (!this.maxRequestSizeSet) { + if (maxRequestSize > 0) { + this.maxRequestSize = maxRequestSize; + this.maxRequestSizeSet = true; + } else { + throw new IllegalArgumentException("'maxRequestSize' must be > 0"); + } + } else { + throw new IllegalArgumentException("'maxRequestSize' can only be set once per instance"); + } + } + + private void assertBytesValid(byte[] bytes) { + if (bytes != null) { + if (bytes.length == 0) { + throw new IllegalArgumentException("'bytes' must not be empty"); + } + } + } + + private void validateInput(InputStream contentStream, String topic, int lastAckedMessageIndex) { + if (contentStream == null) { + throw new IllegalArgumentException("'contentStream' must not be null"); + } else if (topic == null || topic.trim().length() == 0) { + throw new IllegalArgumentException("'topic' must not be null or empty"); + } else if (lastAckedMessageIndex < -1) { + throw new IllegalArgumentException("'lastAckedMessageIndex' must be >= -1"); + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java new file mode 100644 index 0000000000..7874d4de82 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java @@ -0,0 +1,496 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.UUID; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class ConsumeKafkaTest { + + static class MockConsumerPool extends ConsumerPool { + + final int actualMaxLeases; + final List actualTopics; + final Map actualKafkaProperties; + boolean throwKafkaExceptionOnPoll = false; + boolean throwKafkaExceptionOnCommit = false; + Queue> nextPlannedRecordsQueue = new ArrayDeque<>(); + Map nextExpectedCommitOffsets = null; + Map actualCommitOffsets = null; + boolean wasConsumerLeasePoisoned = false; + boolean wasConsumerLeaseClosed = false; + boolean wasPoolClosed = false; + + public MockConsumerPool(int maxLeases, List topics, Map kafkaProperties, ComponentLog logger) { + super(maxLeases, topics, kafkaProperties, null); + actualMaxLeases = maxLeases; + actualTopics = topics; + actualKafkaProperties = kafkaProperties; + } + + @Override + public ConsumerLease obtainConsumer() { + return new ConsumerLease() { + @Override + public ConsumerRecords poll() { + if (throwKafkaExceptionOnPoll) { + throw new KafkaException("i planned to fail"); + } + final ConsumerRecords records = nextPlannedRecordsQueue.poll(); + return (records == null) ? ConsumerRecords.empty() : records; + } + + @Override + public void commitOffsets(Map offsets) { + if (throwKafkaExceptionOnCommit) { + throw new KafkaException("i planned to fail"); + } + actualCommitOffsets = offsets; + } + + @Override + public void poison() { + wasConsumerLeasePoisoned = true; + } + + @Override + public void close() { + wasConsumerLeaseClosed = true; + } + }; + } + + @Override + public void close() { + wasPoolClosed = true; + } + + void resetState() { + throwKafkaExceptionOnPoll = false; + throwKafkaExceptionOnCommit = false; + nextPlannedRecordsQueue = null; + nextExpectedCommitOffsets = null; + wasConsumerLeasePoisoned = false; + wasConsumerLeaseClosed = false; + wasPoolClosed = false; + } + + } + + @Test + public void validateCustomValidatorSettings() throws Exception { + ConsumeKafka consumeKafka = new ConsumeKafka(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo"); + runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + runner.assertValid(); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Foo"); + runner.assertNotValid(); + runner.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + runner.assertValid(); + runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + runner.assertValid(); + runner.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); + runner.assertNotValid(); + } + + @Test + public void validatePropertiesValidation() throws Exception { + ConsumeKafka consumeKafka = new ConsumeKafka(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo"); + runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + + runner.removeProperty(ConsumeKafka.GROUP_ID); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("invalid because group.id is required")); + } + + runner.setProperty(ConsumeKafka.GROUP_ID, ""); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); + } + + runner.setProperty(ConsumeKafka.GROUP_ID, " "); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); + } + } + + @Test + public void validateGetAllMessages() throws Exception { + String groupName = "validateGetAllMessages"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues); + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(firstRecs); + mockPool.nextPlannedRecordsQueue.add(secondRecs); + + ConsumeKafka proc = new ConsumeKafka() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); + + assertEquals(expectedTopics, mockPool.actualTopics); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count()); + + if (mockPool.nextPlannedRecordsQueue.isEmpty()) { + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-5")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-6")).count()); + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset()); + } else { + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + } + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + } + + @Test + public void validateGetLotsOfMessages() throws Exception { + String groupName = "validateGetLotsOfMessages"; + + final byte[][] firstPassValues = new byte[10010][1]; + for (final byte[] value : firstPassValues) { + value[0] = 0x12; + } + final ConsumerRecords firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues); + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(firstRecs); + mockPool.nextPlannedRecordsQueue.add(secondRecs); + + ConsumeKafka proc = new ConsumeKafka() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); + + assertEquals(10010, flowFiles.stream().map(ff -> ff.toByteArray()).filter(content -> content.length == 1 && content[0] == 0x12).count()); + assertEquals(1, mockPool.nextPlannedRecordsQueue.size()); + + assertEquals(1, mockPool.actualCommitOffsets.size()); + assertEquals(10011L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + } + + private ConsumerRecords createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) { + final Map>> map = new HashMap<>(); + final TopicPartition tPart = new TopicPartition(topic, partition); + final List> records = new ArrayList<>(); + long offset = startingOffset; + for (final byte[] rawRecord : rawRecords) { + final ConsumerRecord rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord); + records.add(rec); + } + map.put(tPart, records); + return new ConsumerRecords(map); + } + + private ConsumerRecords mergeRecords(final ConsumerRecords... records) { + final Map>> map = new HashMap<>(); + for (final ConsumerRecords rec : records) { + rec.partitions().stream().forEach((part) -> { + final List> conRecs = rec.records(part); + if (map.get(part) != null) { + throw new IllegalStateException("already have that topic/partition in the record map"); + } + map.put(part, conRecs); + }); + } + return new ConsumerRecords<>(map); + } + + @Test + public void validateGetAllMessagesWithProvidedDemarcator() throws Exception { + String groupName = "validateGetAllMessagesWithProvidedDemarcator"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final byte[][] secondPassValues = new byte[][]{ + "Hello-4".getBytes(StandardCharsets.UTF_8), + "Hello-5".getBytes(StandardCharsets.UTF_8), + "Hello-6".getBytes(StandardCharsets.UTF_8) + }; + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues), + createConsumerRecords("bar", 1, 1L, secondPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + expectedTopics.add("bar"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + + ConsumeKafka proc = new ConsumeKafka() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, false); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); + + assertEquals(2, flowFiles.size()); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count()); + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4blahHello-5blahHello-6")).count()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertFalse(mockPool.wasPoolClosed); + runner.run(1, true); + assertFalse(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + assertEquals(2, mockPool.actualCommitOffsets.size()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset()); + assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset()); + } + + @Test + public void validatePollException() throws Exception { + String groupName = "validatePollException"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + mockPool.throwKafkaExceptionOnPoll = true; + + ConsumeKafka proc = new ConsumeKafka() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo"); + runner.setProperty(ConsumeKafka.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, true); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); + + assertEquals(0, flowFiles.size()); + assertNull(null, mockPool.actualCommitOffsets); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertTrue(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + } + + @Test + public void validateCommitOffsetException() throws Exception { + String groupName = "validateCommitOffsetException"; + + final byte[][] firstPassValues = new byte[][]{ + "Hello-1".getBytes(StandardCharsets.UTF_8), + "Hello-2".getBytes(StandardCharsets.UTF_8), + "Hello-3".getBytes(StandardCharsets.UTF_8) + }; + + final ConsumerRecords consumerRecs = mergeRecords( + createConsumerRecords("foo", 1, 1L, firstPassValues) + ); + + final List expectedTopics = new ArrayList<>(); + expectedTopics.add("foo"); + final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null); + mockPool.nextPlannedRecordsQueue.add(consumerRecs); + mockPool.throwKafkaExceptionOnCommit = true; + + ConsumeKafka proc = new ConsumeKafka() { + @Override + protected ConsumerPool createConsumerPool(final int maxLeases, final List topics, final Map props, final ComponentLog log) { + return mockPool; + } + }; + + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setValidateExpressionUsage(false); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPICS, "foo"); + runner.setProperty(ConsumeKafka.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); + runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah"); + + runner.run(1, true); + + final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); + + assertEquals(1, flowFiles.size()); + + assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count()); + + //asert that all consumers were closed as expected + //assert that the consumer pool was properly closed + assertTrue(mockPool.wasConsumerLeasePoisoned); + assertTrue(mockPool.wasConsumerLeaseClosed); + assertTrue(mockPool.wasPoolClosed); + + assertNull(null, mockPool.actualCommitOffsets); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java new file mode 100644 index 0000000000..7f88ea2b2b --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.util.Collections; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.KafkaException; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processors.kafka.pubsub.ConsumerPool.PoolStats; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConsumerPoolTest { + + Consumer consumer = null; + ComponentLog logger = null; + + @Before + public void setup() { + consumer = mock(Consumer.class); + logger = mock(ComponentLog.class); + } + + @Test + public void validatePoolSimpleCreateClose() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty()); + + try (final ConsumerLease lease = testPool.obtainConsumer()) { + lease.poll(); + lease.commitOffsets(Collections.emptyMap()); + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + assertEquals(1, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } + + @Test + public void validatePoolSimpleBatchCreateClose() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(5, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty()); + + for (int i = 0; i < 100; i++) { + try (final ConsumerLease lease = testPool.obtainConsumer()) { + for (int j = 0; j < 100; j++) { + lease.poll(); + } + lease.commitOffsets(Collections.emptyMap()); + } + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(100, stats.leasesObtainedCount); + assertEquals(10000, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } + + @Test + public void validatePoolConsumerFails() throws Exception { + + final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + + when(consumer.poll(anyInt())).thenThrow(new KafkaException()); + + try (final ConsumerLease lease = testPool.obtainConsumer()) { + lease.poll(); + fail(); + } catch (final KafkaException ke) { + + } + testPool.close(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + assertEquals(0, stats.unproductivePollCount); + assertEquals(0, stats.productivePollCount); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java new file mode 100644 index 0000000000..19c64af1af --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java @@ -0,0 +1,306 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult; +import org.apache.nifi.processors.kafka.test.EmbeddedKafka; +import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import kafka.consumer.Consumer; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.ConsumerTimeoutException; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import org.apache.kafka.clients.producer.ProducerConfig; + +public class KafkaPublisherTest { + + private static EmbeddedKafka kafkaLocal; + + private static EmbeddedKafkaProducerHelper producerHelper; + + @BeforeClass + public static void beforeClass() { + kafkaLocal = new EmbeddedKafka(); + kafkaLocal.start(); + producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal); + } + + @AfterClass + public static void afterClass() throws Exception { + producerHelper.close(); + kafkaLocal.stop(); + } + + @Test + public void validateSuccessfulSendAsWhole() throws Exception { + InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8)); + String topicName = "validateSuccessfulSendAsWhole"; + + Properties kafkaProperties = this.buildProducerProperties(); + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + + PublishingContext publishingContext = new PublishingContext(contentStream, topicName); + KafkaPublisherResult result = publisher.publish(publishingContext); + + assertEquals(0, result.getLastMessageAcked()); + assertEquals(1, result.getMessagesSent()); + contentStream.close(); + publisher.close(); + + ConsumerIterator iter = this.buildConsumer(topicName); + assertNotNull(iter.next()); + try { + iter.next(); + } catch (ConsumerTimeoutException e) { + // that's OK since this is the Kafka mechanism to unblock + } + } + + @Test + public void validateSuccessfulSendAsDelimited() throws Exception { + InputStream contentStream = new ByteArrayInputStream( + "Hello Kafka\nHello Kafka\nHello Kafka\nHello Kafka\n".getBytes(StandardCharsets.UTF_8)); + String topicName = "validateSuccessfulSendAsDelimited"; + + Properties kafkaProperties = this.buildProducerProperties(); + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + + PublishingContext publishingContext = new PublishingContext(contentStream, topicName); + publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8)); + KafkaPublisherResult result = publisher.publish(publishingContext); + + assertEquals(3, result.getLastMessageAcked()); + assertEquals(4, result.getMessagesSent()); + contentStream.close(); + publisher.close(); + + ConsumerIterator iter = this.buildConsumer(topicName); + assertNotNull(iter.next()); + assertNotNull(iter.next()); + assertNotNull(iter.next()); + assertNotNull(iter.next()); + try { + iter.next(); + fail(); + } catch (ConsumerTimeoutException e) { + // that's OK since this is the Kafka mechanism to unblock + } + } + + /* + * This test simulates the condition where not all messages were ACKed by + * Kafka + */ + @Test + public void validateRetries() throws Exception { + byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8); + InputStream contentStream = new ByteArrayInputStream(testValue); + String topicName = "validateSuccessfulReSendOfFailedSegments"; + + Properties kafkaProperties = this.buildProducerProperties(); + + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + + // simulates the first re-try + int lastAckedMessageIndex = 1; + PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex); + publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8)); + + publisher.publish(publishingContext); + + ConsumerIterator iter = this.buildConsumer(topicName); + String m1 = new String(iter.next().message()); + String m2 = new String(iter.next().message()); + assertEquals("Hello Kafka3", m1); + assertEquals("Hello Kafka4", m2); + try { + iter.next(); + fail(); + } catch (ConsumerTimeoutException e) { + // that's OK since this is the Kafka mechanism to unblock + } + + // simulates the second re-try + lastAckedMessageIndex = 2; + contentStream = new ByteArrayInputStream(testValue); + publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex); + publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8)); + publisher.publish(publishingContext); + + m1 = new String(iter.next().message()); + assertEquals("Hello Kafka4", m1); + + publisher.close(); + } + + /* + * Similar to the above test, but it sets the first retry index to the last + * possible message index and second index to an out of bound index. The + * expectation is that no messages will be sent to Kafka + */ + @Test + public void validateRetriesWithWrongIndex() throws Exception { + byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8); + InputStream contentStream = new ByteArrayInputStream(testValue); + String topicName = "validateRetriesWithWrongIndex"; + + Properties kafkaProperties = this.buildProducerProperties(); + + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + + // simulates the first re-try + int lastAckedMessageIndex = 3; + PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex); + publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8)); + + publisher.publish(publishingContext); + + ConsumerIterator iter = this.buildConsumer(topicName); + try { + iter.next(); + fail(); + } catch (ConsumerTimeoutException e) { + // that's OK since this is the Kafka mechanism to unblock + } + + // simulates the second re-try + lastAckedMessageIndex = 6; + contentStream = new ByteArrayInputStream(testValue); + publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex); + publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8)); + publisher.publish(publishingContext); + try { + iter.next(); + fail(); + } catch (ConsumerTimeoutException e) { + // that's OK since this is the Kafka mechanism to unblock + } + + publisher.close(); + } + + @Test + public void validateWithMultiByteCharactersNoDelimiter() throws Exception { + String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE."; + InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8)); + String topicName = "validateWithMultiByteCharacters"; + + Properties kafkaProperties = this.buildProducerProperties(); + + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + PublishingContext publishingContext = new PublishingContext(contentStream, topicName); + + publisher.publish(publishingContext); + publisher.close(); + + ConsumerIterator iter = this.buildConsumer(topicName); + String r = new String(iter.next().message(), StandardCharsets.UTF_8); + assertEquals(data, r); + } + + @Test + public void validateWithNonDefaultPartitioner() throws Exception { + String data = "fooandbarandbaz"; + InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8)); + String topicName = "validateWithNonDefaultPartitioner"; + + Properties kafkaProperties = this.buildProducerProperties(); + kafkaProperties.setProperty("partitioner.class", TestPartitioner.class.getName()); + KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class)); + PublishingContext publishingContext = new PublishingContext(contentStream, topicName); + publishingContext.setDelimiterBytes("and".getBytes(StandardCharsets.UTF_8)); + + try { + publisher.publish(publishingContext); + // partitioner should be invoked 3 times + assertTrue(TestPartitioner.counter == 3); + publisher.close(); + } finally { + TestPartitioner.counter = 0; + } + } + + private Properties buildProducerProperties() { + Properties kafkaProperties = new Properties(); + kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort()); + kafkaProperties.put("auto.create.topics.enable", "true"); + return kafkaProperties; + } + + private ConsumerIterator buildConsumer(String topic) { + Properties props = new Properties(); + props.put("zookeeper.connect", "localhost:" + kafkaLocal.getZookeeperPort()); + props.put("group.id", "test"); + props.put("consumer.timeout.ms", "500"); + props.put("auto.offset.reset", "smallest"); + ConsumerConfig consumerConfig = new ConsumerConfig(props); + ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig); + Map topicCountMap = new HashMap<>(1); + topicCountMap.put(topic, 1); + Map>> consumerMap = consumer.createMessageStreams(topicCountMap); + List> streams = consumerMap.get(topic); + ConsumerIterator iter = streams.get(0).iterator(); + return iter; + } + + public static class TestPartitioner implements Partitioner { + + static int counter; + + @Override + public void configure(Map configs) { + // nothing to do, test + } + + @Override + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, + Cluster cluster) { + counter++; + return 0; + } + + @Override + public void close() { + counter = 0; + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java similarity index 86% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java index 01c5fdd80c..07ae2da106 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java @@ -16,16 +16,9 @@ */ package org.apache.nifi.processors.kafka.pubsub; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - import java.nio.charset.StandardCharsets; - import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.nifi.util.MockFlowFile; @@ -33,22 +26,25 @@ import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; import org.mockito.Mockito; +import static org.mockito.Mockito.times; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.verify; -// The test is valid and should be ran when working on this module. @Ignore is -// to speed up the overall build public class PublishKafkaTest { @Test public void validateCustomSerilaizerDeserializerSettings() throws Exception { PublishKafka publishKafka = new PublishKafka(); TestRunner runner = TestRunners.newTestRunner(publishKafka); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); runner.setProperty(PublishKafka.TOPIC, "foo"); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.META_WAIT_TIME, "3 sec"); - runner.setProperty("key.serializer", ByteArraySerializer.class.getName()); + runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); runner.assertValid(); - runner.setProperty("key.serializer", "Foo"); + runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo"); runner.assertNotValid(); } @@ -56,9 +52,8 @@ public class PublishKafkaTest { public void validatePropertiesValidation() throws Exception { PublishKafka publishKafka = new PublishKafka(); TestRunner runner = TestRunners.newTestRunner(publishKafka); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); runner.setProperty(PublishKafka.TOPIC, "foo"); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.META_WAIT_TIME, "foo"); try { @@ -80,9 +75,8 @@ public class PublishKafkaTest { */ TestRunner runner = TestRunners.newTestRunner(publishKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); - runner.setProperty(PublishKafka.SECURITY_PROTOCOL, PublishKafka.SEC_SASL_PLAINTEXT); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT); try { runner.run(); fail(); @@ -99,9 +93,8 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(100); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8)); @@ -119,9 +112,8 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(100); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName()); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo"); @@ -141,9 +133,8 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(1); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName()); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo"); @@ -164,9 +155,8 @@ public class PublishKafkaTest { TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); runner.setProperty(PublishKafka.META_WAIT_TIME, "3000 millis"); @@ -190,9 +180,8 @@ public class PublishKafkaTest { TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis"); @@ -215,9 +204,8 @@ public class PublishKafkaTest { TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis"); @@ -244,9 +232,8 @@ public class PublishKafkaTest { TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis"); @@ -273,8 +260,7 @@ public class PublishKafkaTest { final TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); runner.setProperty(PublishKafka.KEY, "key1"); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n"); final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8); @@ -292,8 +278,7 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(100); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>".getBytes(StandardCharsets.UTF_8)); @@ -311,8 +296,7 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(100); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>僠<僠WILDSTUFF僠>僠".getBytes(StandardCharsets.UTF_8)); @@ -331,8 +315,7 @@ public class PublishKafkaTest { StubPublishKafka putKafka = new StubPublishKafka(100); TestRunner runner = TestRunners.newTestRunner(putKafka); runner.setProperty(PublishKafka.TOPIC, topicName); - runner.setProperty(PublishKafka.CLIENT_ID, "foo"); - runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234"); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234"); runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠"); runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDBOOMSTUFF僠>僠".getBytes(StandardCharsets.UTF_8)); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java new file mode 100644 index 0000000000..4a9a1c07ba --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; + +import java.io.InputStream; +import java.nio.charset.StandardCharsets; + +import org.junit.Test; + +public class PublishingContextTest { + + @Test + public void failInvalidConstructorArgs() { + try { + new PublishingContext(null, null); + fail(); + } catch (IllegalArgumentException e) { + // success + } + try { + new PublishingContext(mock(InputStream.class), null); + fail(); + } catch (IllegalArgumentException e) { + // success + } + + try { + new PublishingContext(mock(InputStream.class), ""); + fail(); + } catch (IllegalArgumentException e) { + // success + } + + try { + new PublishingContext(mock(InputStream.class), "mytopic", -3); + fail(); + } catch (IllegalArgumentException e) { + // success + } + } + + @Test + public void validateFullSetting() { + PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic", 3); + publishingContext.setDelimiterBytes("delimiter".getBytes(StandardCharsets.UTF_8)); + publishingContext.setKeyBytes("key".getBytes(StandardCharsets.UTF_8)); + + assertEquals("delimiter", new String(publishingContext.getDelimiterBytes(), StandardCharsets.UTF_8)); + assertEquals("key", new String(publishingContext.getKeyBytes(), StandardCharsets.UTF_8)); + assertEquals("topic", publishingContext.getTopic()); + assertEquals("topic: 'topic'; delimiter: 'delimiter'", publishingContext.toString()); + } + + @Test + public void validateOnlyOnceSetPerInstance() { + PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic"); + publishingContext.setKeyBytes(new byte[]{0}); + try { + publishingContext.setKeyBytes(new byte[]{0}); + fail(); + } catch (IllegalArgumentException e) { + // success + } + + publishingContext.setDelimiterBytes(new byte[]{0}); + try { + publishingContext.setDelimiterBytes(new byte[]{0}); + fail(); + } catch (IllegalArgumentException e) { + // success + } + + publishingContext.setMaxRequestSize(1024); + try { + publishingContext.setMaxRequestSize(1024); + fail(); + } catch (IllegalArgumentException e) { + // success + } + + try { + publishingContext.setMaxRequestSize(-10); + fail(); + } catch (IllegalArgumentException e) { + // success + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java similarity index 94% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java index 3189356c01..950d623c65 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java @@ -16,12 +16,12 @@ */ package org.apache.nifi.processors.kafka.pubsub; -import static org.mockito.Mockito.mock; +import java.lang.reflect.Field; import static org.mockito.Mockito.when; -import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; -import java.util.Properties; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -38,7 +38,9 @@ import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.exception.ProcessException; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS; import org.mockito.Mockito; +import static org.mockito.Mockito.mock; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -68,7 +70,8 @@ public class StubPublishKafka extends PublishKafka { @Override protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException { - Properties kafkaProperties = this.buildKafkaProperties(context); + final Map kafkaProperties = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties); kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); KafkaPublisher publisher; diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java new file mode 100644 index 0000000000..819e3b73ac --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.pubsub; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; + +import sun.misc.Unsafe; + +class TestUtils { + + public static void setFinalField(Field field, Object instance, Object newValue) throws Exception { + field.setAccessible(true); + Field modifiersField = Field.class.getDeclaredField("modifiers"); + modifiersField.setAccessible(true); + modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL); + + field.set(instance, newValue); + } + + static Unsafe getUnsafe() { + try { + Field f = Unsafe.class.getDeclaredField("theUnsafe"); + f.setAccessible(true); + return (Unsafe) f.get(null); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java new file mode 100644 index 0000000000..802f8891fb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.test; + +import java.io.File; +import java.io.IOException; +import java.net.ServerSocket; +import java.util.Properties; + +import org.apache.commons.io.FileUtils; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ServerConfig; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; + +/** + * Embedded Kafka server, primarily to be used for testing. + */ +public class EmbeddedKafka { + + private final KafkaServerStartable kafkaServer; + + private final Properties zookeeperConfig; + + private final Properties kafkaConfig; + + private final ZooKeeperServer zkServer; + + private final Logger logger = LoggerFactory.getLogger(EmbeddedKafka.class); + + private final int kafkaPort; + + private final int zookeeperPort; + + private boolean started; + + /** + * Will create instance of the embedded Kafka server. Kafka and Zookeeper + * configuration properties will be loaded from 'server.properties' and + * 'zookeeper.properties' located at the root of the classpath. + */ + public EmbeddedKafka() { + this(loadPropertiesFromClasspath("/server.properties"), loadPropertiesFromClasspath("/zookeeper.properties")); + } + + /** + * Will create instance of the embedded Kafka server. + * + * @param kafkaConfig + * Kafka configuration properties + * @param zookeeperConfig + * Zookeeper configuration properties + */ + public EmbeddedKafka(Properties kafkaConfig, Properties zookeeperConfig) { + this.cleanupKafkaWorkDir(); + this.zookeeperConfig = zookeeperConfig; + this.kafkaConfig = kafkaConfig; + this.kafkaPort = this.availablePort(); + this.zookeeperPort = this.availablePort(); + + this.kafkaConfig.setProperty("port", String.valueOf(this.kafkaPort)); + this.kafkaConfig.setProperty("zookeeper.connect", "localhost:" + this.zookeeperPort); + this.zookeeperConfig.setProperty("clientPort", String.valueOf(this.zookeeperPort)); + this.zkServer = new ZooKeeperServer(); + this.kafkaServer = new KafkaServerStartable(new KafkaConfig(kafkaConfig)); + } + + /** + * + * @return port for Kafka server + */ + public int getKafkaPort() { + if (!this.started) { + throw new IllegalStateException("Kafka server is not started. Kafka port can't be determined."); + } + return this.kafkaPort; + } + + /** + * + * @return port for Zookeeper server + */ + public int getZookeeperPort() { + if (!this.started) { + throw new IllegalStateException("Kafka server is not started. Zookeeper port can't be determined."); + } + return this.zookeeperPort; + } + + /** + * Will start embedded Kafka server. Its data directories will be created + * at 'kafka-tmp' directory relative to the working directory of the current + * runtime. The data directories will be deleted upon JVM exit. + * + */ + public void start() { + if (!this.started) { + logger.info("Starting Zookeeper server"); + this.startZookeeper(); + + logger.info("Starting Kafka server"); + this.kafkaServer.startup(); + + logger.info("Embeded Kafka is started at localhost:" + this.kafkaServer.serverConfig().port() + + ". Zookeeper connection string: " + this.kafkaConfig.getProperty("zookeeper.connect")); + this.started = true; + } + } + + /** + * Will stop embedded Kafka server, cleaning up all working directories. + */ + public void stop() { + if (this.started) { + logger.info("Shutting down Kafka server"); + this.kafkaServer.shutdown(); + this.kafkaServer.awaitShutdown(); + logger.info("Shutting down Zookeeper server"); + this.shutdownZookeeper(); + logger.info("Embeded Kafka is shut down."); + this.cleanupKafkaWorkDir(); + this.started = false; + } + } + + /** + * + */ + private void cleanupKafkaWorkDir() { + File kafkaTmp = new File("target/kafka-tmp"); + try { + FileUtils.deleteDirectory(kafkaTmp); + } catch (Exception e) { + logger.warn("Failed to delete " + kafkaTmp.getAbsolutePath()); + } + } + + /** + * Will start Zookeeper server via {@link ServerCnxnFactory} + */ + private void startZookeeper() { + QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig(); + try { + quorumConfiguration.parseProperties(this.zookeeperConfig); + + ServerConfig configuration = new ServerConfig(); + configuration.readFrom(quorumConfiguration); + + FileTxnSnapLog txnLog = new FileTxnSnapLog(new File(configuration.getDataLogDir()), new File(configuration.getDataDir())); + + zkServer.setTxnLogFactory(txnLog); + zkServer.setTickTime(configuration.getTickTime()); + zkServer.setMinSessionTimeout(configuration.getMinSessionTimeout()); + zkServer.setMaxSessionTimeout(configuration.getMaxSessionTimeout()); + ServerCnxnFactory zookeeperConnectionFactory = ServerCnxnFactory.createFactory(); + zookeeperConnectionFactory.configure(configuration.getClientPortAddress(), + configuration.getMaxClientCnxns()); + zookeeperConnectionFactory.startup(zkServer); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new IllegalStateException("Failed to start Zookeeper server", e); + } + } + + /** + * Will shut down Zookeeper server. + */ + private void shutdownZookeeper() { + zkServer.shutdown(); + } + + /** + * Will load {@link Properties} from properties file discovered at the + * provided path relative to the root of the classpath. + */ + private static Properties loadPropertiesFromClasspath(String path) { + try { + Properties kafkaProperties = new Properties(); + kafkaProperties.load(Class.class.getResourceAsStream(path)); + return kafkaProperties; + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + /** + * Will determine the available port used by Kafka/Zookeeper servers. + */ + private int availablePort() { + ServerSocket s = null; + try { + s = new ServerSocket(0); + s.setReuseAddress(true); + return s.getLocalPort(); + } catch (Exception e) { + throw new IllegalStateException("Failed to discover available port.", e); + } finally { + try { + s.close(); + } catch (IOException e) { + // ignore + } + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java new file mode 100644 index 0000000000..0ed00fb5c3 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafkaProducerHelper.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.kafka.test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Properties; + +import kafka.producer.KeyedMessage; +import kafka.producer.OldProducer; + +/** + * Helper class which helps to produce events targeting {@link EmbeddedKafka} + * server. + */ +public class EmbeddedKafkaProducerHelper implements Closeable { + + private final EmbeddedKafka kafkaServer; + + private final OldProducer producer; + + /** + * Will create an instance of EmbeddedKafkaProducerHelper based on default + * configurations.
+ * Default configuration includes:
+ * + * metadata.broker.list=[determined from the instance of EmbeddedKafka]
+ * serializer.class=kafka.serializer.DefaultEncoder
+ * key.serializer.class=kafka.serializer.DefaultEncoder
+ * auto.create.topics.enable=true + *

+ *
+ * If you wish to supply additional configuration properties or override + * existing use + * {@link EmbeddedKafkaProducerHelper#EmbeddedKafkaProducerHelper(EmbeddedKafka, Properties)} + * constructor. + * + * @param kafkaServer + * instance of {@link EmbeddedKafka} + */ + public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer) { + this(kafkaServer, null); + } + + /** + * Will create an instance of EmbeddedKafkaProducerHelper based on default + * configurations and additional configuration properties.
+ * Default configuration includes:
+ * metadata.broker.list=[determined from the instance of EmbeddedKafka]
+ * serializer.class=kafka.serializer.DefaultEncoder
+ * key.serializer.class=kafka.serializer.DefaultEncoder
+ * auto.create.topics.enable=true
+ *
+ * + * @param kafkaServer + * instance of {@link EmbeddedKafka} + * @param additionalProperties + * instance of {@link Properties} specifying additional producer + * configuration properties. + */ + public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer, Properties additionalProperties) { + this.kafkaServer = kafkaServer; + Properties producerProperties = new Properties(); + producerProperties.put("metadata.broker.list", "localhost:" + this.kafkaServer.getKafkaPort()); + producerProperties.put("serializer.class", "kafka.serializer.DefaultEncoder"); + producerProperties.put("key.serializer.class", "kafka.serializer.DefaultEncoder"); + producerProperties.put("auto.create.topics.enable", "true"); + if (additionalProperties != null) { + producerProperties.putAll(additionalProperties); + } + this.producer = new OldProducer(producerProperties); + } + + /** + * Will send an event to a Kafka topic. If topic doesn't exist it will be + * auto-created. + * + * @param topicName + * Kafka topic name. + * @param event + * string representing an event(message) to be sent to Kafka. + */ + public void sendEvent(String topicName, String event) { + KeyedMessage data = new KeyedMessage(topicName, event.getBytes()); + this.producer.send(data.topic(), data.key(), data.message()); + } + + /** + * Will close the underlying Kafka producer. + */ + @Override + public void close() throws IOException { + this.producer.close(); + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/log4j.properties new file mode 100644 index 0000000000..57cd63f139 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/log4j.properties @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +log4j.rootCategory=INFO, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %t %c{2}:%L - %m%n + +#og4j.category.org.apache.nifi.processors.kafka=DEBUG diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/server.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/server.properties new file mode 100644 index 0000000000..2ecb1b20ba --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/server.properties @@ -0,0 +1,121 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +#port=9092 + +# Hostname the broker will bind to. If not set, the server will bind to all interfaces +#host.name=localhost + +# Hostname the broker will advertise to producers and consumers. If not set, it uses the +# value for "host.name" if configured. Otherwise, it will use the value returned from +# java.net.InetAddress.getCanonicalHostName(). +#advertised.host.name= + +# The port to publish to ZooKeeper for clients to use. If this is not set, +# it will publish the same port that the broker binds to. +#advertised.port= + +# The number of threads handling network requests +num.network.threads=3 + +# The number of threads doing disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma seperated list of directories under which to store log files +log.dirs=target/kafka-tmp/kafka-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +############################# Zookeeper ############################# + +# Zookeeper connection string (see zookeeper docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=6000 diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/zookeeper.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/zookeeper.properties new file mode 100644 index 0000000000..f5c257efeb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/resources/zookeeper.properties @@ -0,0 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# the directory where the snapshot is stored. +dataDir=target/kafka-tmp/zookeeper +# the port at which the clients will connect +#clientPort=2181 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java deleted file mode 100644 index 4677e33a00..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java +++ /dev/null @@ -1,400 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import java.io.Closeable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Pattern; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.components.AllowableValue; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.PropertyDescriptor.Builder; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.processor.AbstractSessionFactoryProcessor; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.Processor; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.ssl.SSLContextService; -import org.apache.nifi.util.FormatUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Base class for implementing {@link Processor}s to publish and consume - * messages to/from Kafka - * - * @see PublishKafka - * @see ConsumeKafka - */ -abstract class AbstractKafkaProcessor extends AbstractSessionFactoryProcessor { - - final Logger logger = LoggerFactory.getLogger(this.getClass()); - - private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}"; - - private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*"; - - - static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT"); - static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL"); - static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT"); - static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL"); - - static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder() - .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) - .displayName("Kafka Brokers") - .description("A comma-separated list of known Kafka Brokers in the format :") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) - .expressionLanguageSupported(true) - .defaultValue("localhost:9092") - .build(); - static final PropertyDescriptor CLIENT_ID = new PropertyDescriptor.Builder() - .name(ProducerConfig.CLIENT_ID_CONFIG) - .displayName("Client ID") - .description("String value uniquely identifying this client application. Corresponds to Kafka's 'client.id' property.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(true) - .build(); - static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder() - .name("security.protocol") - .displayName("Security Protocol") - .description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.") - .required(false) - .expressionLanguageSupported(false) - .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL) - .defaultValue(SEC_PLAINTEXT.getValue()) - .build(); - static final PropertyDescriptor KERBEROS_PRINCIPLE = new PropertyDescriptor.Builder() - .name("sasl.kerberos.service.name") - .displayName("Kerberos Service Name") - .description("The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config. " - + "Corresponds to Kafka's 'security.protocol' property." - + "It is ignored unless one of the SASL options of the are selected.") - .required(false) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(false) - .build(); - - static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder() - .name("topic") - .displayName("Topic Name") - .description("The name of the Kafka Topic") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(true) - .build(); - - static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() - .name("ssl.context.service") - .displayName("SSL Context Service") - .description("Specifies the SSL Context Service to use for communicating with Kafka.") - .required(false) - .identifiesControllerService(SSLContextService.class) - .build(); - - static final Builder MESSAGE_DEMARCATOR_BUILDER = new PropertyDescriptor.Builder() - .name("message-demarcator") - .displayName("Message Demarcator") - .required(false) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true); - - static final Relationship REL_SUCCESS = new Relationship.Builder() - .name("success") - .description("All FlowFiles that are the are successfully sent to or received from Kafka are routed to this relationship") - .build(); - - static final List SHARED_DESCRIPTORS = new ArrayList<>(); - - static final Set SHARED_RELATIONSHIPS = new HashSet<>(); - - private final AtomicInteger taskCounter = new AtomicInteger(); - - private volatile boolean acceptTask = true; - - static { - SHARED_DESCRIPTORS.add(BOOTSTRAP_SERVERS); - SHARED_DESCRIPTORS.add(TOPIC); - SHARED_DESCRIPTORS.add(CLIENT_ID); - SHARED_DESCRIPTORS.add(SECURITY_PROTOCOL); - SHARED_DESCRIPTORS.add(KERBEROS_PRINCIPLE); - SHARED_DESCRIPTORS.add(SSL_CONTEXT_SERVICE); - - SHARED_RELATIONSHIPS.add(REL_SUCCESS); - } - - /** - * Instance of {@link KafkaPublisher} or {@link KafkaConsumer} - */ - volatile T kafkaResource; - - /** - * This thread-safe operation will delegate to - * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first - * checking and creating (if necessary) Kafka resource which could be either - * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and - * destroy the underlying Kafka resource upon catching an {@link Exception} - * raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}. - * After Kafka resource is destroyed it will be re-created upon the next - * invocation of this operation essentially providing a self healing mechanism - * to deal with potentially corrupted resource. - *

- * Keep in mind that upon catching an exception the state of this processor - * will be set to no longer accept any more tasks, until Kafka resource is reset. - * This means that in a multi-threaded situation currently executing tasks will - * be given a chance to complete while no new tasks will be accepted. - */ - @Override - public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException { - if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaResource' can be reset before new tasks are accepted. - this.taskCounter.incrementAndGet(); - final ProcessSession session = sessionFactory.createSession(); - try { - /* - * We can't be doing double null check here since as a pattern - * it only works for lazy init but not reset, which is what we - * are doing here. In fact the first null check is dangerous - * since 'kafkaResource' can become null right after its null - * check passed causing subsequent NPE. - */ - synchronized (this) { - if (this.kafkaResource == null) { - this.kafkaResource = this.buildKafkaResource(context, session); - } - } - - /* - * The 'processed' boolean flag does not imply any failure or success. It simply states that: - * - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated - * - PublishKafka - some messages were sent to Kafka based on existence of the input FlowFile - */ - boolean processed = this.rendezvousWithKafka(context, session); - session.commit(); - if (processed) { - this.postCommit(context); - } else { - context.yield(); - } - } catch (Throwable e) { - this.acceptTask = false; - session.rollback(true); - this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[] { this, e }); - } finally { - synchronized (this) { - if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) { - this.close(); - this.acceptTask = true; - } - } - } - } else { - this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); - this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset."); - context.yield(); - } - } - - /** - * Will call {@link Closeable#close()} on the target resource after which - * the target resource will be set to null. Should only be called when there - * are no more threads being executed on this processor or when it has been - * verified that only a single thread remains. - * - * @see KafkaPublisher - * @see KafkaConsumer - */ - @OnStopped - public void close() { - try { - if (this.kafkaResource != null) { - try { - this.kafkaResource.close(); - } catch (Exception e) { - this.getLogger().warn("Failed while closing " + this.kafkaResource, e); - } - } - } finally { - this.kafkaResource = null; - } - } - - /** - * - */ - @Override - protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { - return new PropertyDescriptor.Builder() - .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.") - .name(propertyDescriptorName).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true) - .build(); - } - - /** - * This operation is called from - * {@link #onTrigger(ProcessContext, ProcessSessionFactory)} method and - * contains main processing logic for this Processor. - */ - protected abstract boolean rendezvousWithKafka(ProcessContext context, ProcessSession session); - - /** - * Builds target resource for interacting with Kafka. The target resource - * could be one of {@link KafkaPublisher} or {@link KafkaConsumer} - */ - protected abstract T buildKafkaResource(ProcessContext context, ProcessSession session); - - /** - * This operation will be executed after {@link ProcessSession#commit()} has - * been called. - */ - protected void postCommit(ProcessContext context) { - // no op - } - - /** - * - */ - @Override - protected Collection customValidate(ValidationContext validationContext) { - List results = new ArrayList<>(); - - String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue(); - - /* - * validates that if one of SASL (Kerberos) option is selected for - * security protocol, then Kerberos principal is provided as well - */ - if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)){ - String kerberosPrincipal = validationContext.getProperty(KERBEROS_PRINCIPLE).getValue(); - if (kerberosPrincipal == null || kerberosPrincipal.trim().length() == 0){ - results.add(new ValidationResult.Builder().subject(KERBEROS_PRINCIPLE.getDisplayName()).valid(false) - .explanation("The <" + KERBEROS_PRINCIPLE.getDisplayName() + "> property must be set when <" - + SECURITY_PROTOCOL.getDisplayName() + "> is configured as '" - + SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.") - .build()); - } - } - - String keySerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).build()) - .getValue(); - if (keySerializer != null && !ByteArraySerializer.class.getName().equals(keySerializer)) { - results.add(new ValidationResult.Builder().subject(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG) - .explanation("Key Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + keySerializer + "'").build()); - } - String valueSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).build()) - .getValue(); - if (valueSerializer != null && !ByteArraySerializer.class.getName().equals(valueSerializer)) { - results.add(new ValidationResult.Builder().subject(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG) - .explanation("Value Serializer must be " + ByteArraySerializer.class.getName() + "' was '" + valueSerializer + "'").build()); - } - String keyDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).build()) - .getValue(); - if (keyDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(keyDeSerializer)) { - results.add(new ValidationResult.Builder().subject(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG) - .explanation("Key De-Serializer must be '" + ByteArrayDeserializer.class.getName() + "' was '" + keyDeSerializer + "'").build()); - } - String valueDeSerializer = validationContext.getProperty(new PropertyDescriptor.Builder().name(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).build()) - .getValue(); - if (valueDeSerializer != null && !ByteArrayDeserializer.class.getName().equals(valueDeSerializer)) { - results.add(new ValidationResult.Builder().subject(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG) - .explanation("Value De-Serializer must be " + ByteArrayDeserializer.class.getName() + "' was '" + valueDeSerializer + "'").build()); - } - - return results; - } - - /** - * Builds transit URI for provenance event. The transit URI will be in the - * form of <security.protocol>://<bootstrap.servers>/topic - */ - String buildTransitURI(String securityProtocol, String brokers, String topic) { - StringBuilder builder = new StringBuilder(); - builder.append(securityProtocol); - builder.append("://"); - builder.append(brokers); - builder.append("/"); - builder.append(topic); - return builder.toString(); - } - - /** - * Builds Kafka {@link Properties} - */ - Properties buildKafkaProperties(ProcessContext context) { - Properties properties = new Properties(); - for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) { - if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) { - // Translate SSLContext Service configuration into Kafka properties - final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); - buildSSLKafkaProperties(sslContextService, properties); - continue; - } - - String pName = propertyDescriptor.getName(); - String pValue = propertyDescriptor.isExpressionLanguageSupported() - ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue() - : context.getProperty(propertyDescriptor).getValue(); - if (pValue != null) { - if (pName.endsWith(".ms")) { // kafka standard time notation - pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS)); - } - properties.setProperty(pName, pValue); - } - } - return properties; - } - - private void buildSSLKafkaProperties(final SSLContextService sslContextService, final Properties properties) { - if (sslContextService == null) { - return; - } - - if (sslContextService.isKeyStoreConfigured()) { - properties.setProperty("ssl.keystore.location", sslContextService.getKeyStoreFile()); - properties.setProperty("ssl.keystore.password", sslContextService.getKeyStorePassword()); - final String keyPass = sslContextService.getKeyPassword() == null ? sslContextService.getKeyStorePassword() : sslContextService.getKeyPassword(); - properties.setProperty("ssl.key.password", keyPass); - properties.setProperty("ssl.keystore.type", sslContextService.getKeyStoreType()); - } - - if (sslContextService.isTrustStoreConfigured()) { - properties.setProperty("ssl.truststore.location", sslContextService.getTrustStoreFile()); - properties.setProperty("ssl.truststore.password", sslContextService.getTrustStorePassword()); - properties.setProperty("ssl.truststore.type", sslContextService.getTrustStoreType()); - } - } -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java deleted file mode 100644 index ac5b4c57c7..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java +++ /dev/null @@ -1,296 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import java.io.IOException; -import java.io.OutputStream; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.nifi.annotation.behavior.InputRequirement; -import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.components.AllowableValue; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.processor.util.StandardValidators; - -@InputRequirement(Requirement.INPUT_FORBIDDEN) -@CapabilityDescription("Consumes messages from Apache Kafka") -@Tags({ "Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume" }) -public class ConsumeKafka extends AbstractKafkaProcessor> { - - static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset"); - - static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset"); - - static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group"); - - static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder() - .name(ConsumerConfig.GROUP_ID_CONFIG) - .displayName("Group ID") - .description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.") - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .expressionLanguageSupported(false) - .build(); - static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder() - .name(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) - .displayName("Offset Reset") - .description("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any " - + "more on the server (e.g. because that data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.") - .required(true) - .allowableValues(OFFSET_EARLIEST, OFFSET_LATEST, OFFSET_NONE) - .defaultValue(OFFSET_LATEST.getValue()) - .build(); - static final PropertyDescriptor MESSAGE_DEMARCATOR = MESSAGE_DEMARCATOR_BUILDER - .description("Since KafkaConsumer receives messages in batches, you have an option to output a single FlowFile which contains " - + "all Kafka messages in a single batch and this property allows you to provide a string (interpreted as UTF-8) to use " - + "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received " - + "in a batch will result in a single FlowFile which essentially means that this processor may output multiple FlowFiles for each " - + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS") - .build(); - - - static final List DESCRIPTORS; - - static final Set RELATIONSHIPS; - - private volatile byte[] demarcatorBytes; - - private volatile String topic; - - private volatile String brokers; - - /* - * Will ensure that the list of the PropertyDescriptors is build only once, - * since all other lifecycle methods are invoked multiple times. - */ - static { - List _descriptors = new ArrayList<>(); - _descriptors.addAll(SHARED_DESCRIPTORS); - _descriptors.add(GROUP_ID); - _descriptors.add(AUTO_OFFSET_RESET); - _descriptors.add(MESSAGE_DEMARCATOR); - DESCRIPTORS = Collections.unmodifiableList(_descriptors); - - RELATIONSHIPS = Collections.unmodifiableSet(SHARED_RELATIONSHIPS); - } - - /** - * - */ - @Override - public Set getRelationships() { - return RELATIONSHIPS; - } - - /** - * Will unsubscribe form {@link KafkaConsumer} delegating to 'super' to do - * the rest. - */ - @Override - @OnStopped - public void close() { - if (this.kafkaResource != null) { - try { - this.kafkaResource.unsubscribe(); - } finally { // in the event the above fails - super.close(); - } - } - } - - /** - * - */ - @Override - protected List getSupportedPropertyDescriptors() { - return DESCRIPTORS; - } - - /** - * Will rendezvous with Kafka by performing the following: - *
- * - poll {@link ConsumerRecords} from {@link KafkaConsumer} in a - * non-blocking manner, signaling yield if no records were received from - * Kafka - *
- * - if records were received form Kafka, the are written to a newly created - * {@link FlowFile}'s {@link OutputStream} using a provided demarcator (see - * {@link #MESSAGE_DEMARCATOR} - */ - @Override - protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession processSession) { - ConsumerRecords consumedRecords = this.kafkaResource.poll(100); - if (consumedRecords != null && !consumedRecords.isEmpty()) { - long start = System.nanoTime(); - FlowFile flowFile = processSession.create(); - final AtomicInteger messageCounter = new AtomicInteger(); - final Map kafkaAttributes = new HashMap<>(); - - final Iterator> iter = consumedRecords.iterator(); - while (iter.hasNext()){ - flowFile = processSession.append(flowFile, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - ConsumerRecord consumedRecord = iter.next(); - - kafkaAttributes.put("kafka.offset", String.valueOf(consumedRecord.offset())); - if (consumedRecord.key() != null) { - kafkaAttributes.put("kafka.key", new String(consumedRecord.key(), StandardCharsets.UTF_8)); - } - kafkaAttributes.put("kafka.partition", String.valueOf(consumedRecord.partition())); - kafkaAttributes.put("kafka.topic", consumedRecord.topic()); - - if (messageCounter.getAndIncrement() > 0 && ConsumeKafka.this.demarcatorBytes != null) { - out.write(ConsumeKafka.this.demarcatorBytes); - } - out.write(consumedRecord.value()); - } - }); - - flowFile = processSession.putAllAttributes(flowFile, kafkaAttributes); - /* - * Release FlowFile if there are more messages in the - * ConsumerRecords batch and no demarcator was provided, - * otherwise the FlowFile will be released as soon as this loop - * exits. - */ - if (iter.hasNext() && ConsumeKafka.this.demarcatorBytes == null){ - this.releaseFlowFile(flowFile, context, processSession, start, messageCounter.get()); - flowFile = processSession.create(); - messageCounter.set(0); - } - } - this.releaseFlowFile(flowFile, context, processSession, start, messageCounter.get()); - } - return consumedRecords != null && !consumedRecords.isEmpty(); - } - - /** - * This operation is called from - * {@link #onTrigger(ProcessContext, ProcessSessionFactory)} method after - * the process session is committed so that then kafka offset changes can be - * committed. This can mean in cases of really bad timing we could have data - * duplication upon recovery but not data loss. We want to commit the flow - * files in a NiFi sense before we commit them in a Kafka sense. - */ - @Override - protected void postCommit(ProcessContext context) { - this.kafkaResource.commitSync(); - } - - /** - * Builds and instance of {@link KafkaConsumer} and subscribes to a provided - * topic. - */ - @Override - protected Consumer buildKafkaResource(ProcessContext context, ProcessSession session) { - this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() - ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) - : null; - this.topic = context.getProperty(TOPIC).evaluateAttributeExpressions().getValue(); - this.brokers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); - Properties kafkaProperties = this.buildKafkaProperties(context); - - /* - * Since we are using unconventional way to validate if connectivity to - * broker is possible we need a mechanism to be able to disable it. - * 'check.connection' property will serve as such mechanism - */ - if (!"false".equals(kafkaProperties.get("check.connection"))) { - this.checkIfInitialConnectionPossible(); - } - - kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - - KafkaConsumer consumer = new KafkaConsumer<>(kafkaProperties); - consumer.subscribe(Collections.singletonList(this.topic)); - return consumer; - } - - /** - * Checks via brute force if it is possible to establish connection to at - * least one broker. If not this method will throw {@link ProcessException}. - */ - private void checkIfInitialConnectionPossible(){ - String[] br = this.brokers.split(","); - boolean connectionPossible = false; - for (int i = 0; i < br.length && !connectionPossible; i++) { - String hostPortCombo = br[i]; - String[] hostPort = hostPortCombo.split(":"); - Socket client = null; - try { - client = new Socket(); - client.connect(new InetSocketAddress(hostPort[0].trim(), Integer.parseInt(hostPort[1].trim())), 10000); - connectionPossible = true; - } catch (Exception e) { - this.logger.error("Connection to '" + hostPortCombo + "' is not possible", e); - } finally { - try { - client.close(); - } catch (IOException e) { - // ignore - } - } - } - if (!connectionPossible){ - throw new ProcessException("Connection to " + this.brokers + " is not possible. See logs for more details"); - } - } - /** - * Will release flow file. Releasing of the flow file in the context of this - * operation implies the following: - * - * If Empty then remove from session and return If has something then - * transfer to {@link #REL_SUCCESS} - */ - private void releaseFlowFile(FlowFile flowFile, ProcessContext context, ProcessSession session, long start, int msgCount) { - long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); - String transitUri = this.buildTransitURI(context.getProperty(SECURITY_PROTOCOL).getValue(), this.brokers, topic); - session.getProvenanceReporter().receive(flowFile, transitUri, "Received " + msgCount + " Kafka messages", executionDuration); - this.getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[] { flowFile, msgCount, executionDuration }); - session.transfer(flowFile, REL_SUCCESS); - } -} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java deleted file mode 100644 index e2cdea29dd..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.nifi.logging.ComponentLog; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collections; -import java.util.Properties; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -/** - * A pool of Kafka Consumers for a given topic. Clients must create the ConsumerPool and call initialize() before - * acquiring consumers. Consumers should be returned by calling returnConsumerResource. - */ -public class ConsumerPool implements Closeable { - - private final int size; - private final BlockingQueue consumers; - private final String topic; - private final Properties kafkaProperties; - private final ComponentLog logger; - private boolean initialized = false; - - /** - * Initializes the pool with the given size, topic, properties, and logger, but does not create any consumers until initialize() is called. - * - * @param size the number of consumers to pool - * @param topic the topic to consume from - * @param kafkaProperties the properties for each consumer - * @param logger the logger to report any errors/warnings - */ - public ConsumerPool(final int size, final String topic, final Properties kafkaProperties, final ComponentLog logger) { - this.size = size; - this.logger = logger; - this.topic = topic; - this.kafkaProperties = kafkaProperties; - this.consumers = new LinkedBlockingQueue<>(size); - } - - /** - * Creates the consumers and subscribes them to the given topic. This method must be called before - * acquiring any consumers. - */ - public synchronized void initialize() { - if (initialized) { - return; - } - - for (int i=0; i < size; i++) { - ConsumerResource resource = createConsumerResource(); - consumers.offer(resource); - } - - initialized = true; - } - - /** - * @return a ConsumerResource from the pool, or a newly created ConsumerResource if none were available in the pool - * @throws IllegalStateException if attempting to get a consumer before calling initialize() - */ - public synchronized ConsumerResource getConsumerResource() { - if (!initialized) { - throw new IllegalStateException("ConsumerPool must be initialized before acquiring consumers"); - } - - ConsumerResource consumerResource = consumers.poll(); - if (consumerResource == null) { - consumerResource = createConsumerResource(); - } - return consumerResource; - } - - /** - * If the given ConsumerResource has been poisoned then it is closed and not returned to the pool, - * otherwise it is attempted to be returned to the pool. If the pool is already full then it is closed - * and not returned. - * - * @param consumerResource - */ - public synchronized void returnConsumerResource(final ConsumerResource consumerResource) { - if (consumerResource == null) { - return; - } - - if (consumerResource.isPoisoned()) { - closeConsumer(consumerResource.getConsumer()); - } else { - boolean added = consumers.offer(consumerResource); - if (!added) { - closeConsumer(consumerResource.getConsumer()); - } - } - } - - /** - * Closes all ConsumerResources in the pool and resets the initialization state of this pool. - * - * @throws IOException should never throw - */ - @Override - public synchronized void close() throws IOException { - ConsumerResource consumerResource; - while ((consumerResource = consumers.poll()) != null) { - closeConsumer(consumerResource.getConsumer()); - } - initialized = false; - } - - private ConsumerResource createConsumerResource() { - final Consumer kafkaConsumer = new KafkaConsumer<>(kafkaProperties); - kafkaConsumer.subscribe(Collections.singletonList(this.topic)); - return new ConsumerResource(kafkaConsumer, this, logger); - } - - private void closeConsumer(Consumer consumer) { - try { - consumer.unsubscribe(); - } catch (Exception e) { - logger.warn("Failed while unsubscribing " + consumer, e); - } - - try { - consumer.close(); - } catch (Exception e) { - logger.warn("Failed while closing " + consumer, e); - } - } - -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerResource.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerResource.java deleted file mode 100644 index baaf39fd0c..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerResource.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.nifi.logging.ComponentLog; - -import java.io.Closeable; -import java.io.IOException; - -/** - * A wrapper for a Kafka Consumer obtained from a ConsumerPool. Client's should call poison() to indicate that this - * consumer should no longer be used by other clients, and should always call close(). Calling close() will pass - * this consumer back to the pool and the pool will determine the appropriate handling based on whether the consumer - * has been poisoned and whether the pool is already full. - */ -public class ConsumerResource implements Closeable { - - private final ComponentLog logger; - private final Consumer consumer; - private final ConsumerPool consumerPool; - private boolean poisoned = false; - - /** - * @param consumer the Kafka Consumer - * @param consumerPool the ConsumerPool this ConsumerResource was obtained from - * @param logger the logger to report any errors/warnings - */ - public ConsumerResource(Consumer consumer, ConsumerPool consumerPool, ComponentLog logger) { - this.logger = logger; - this.consumer = consumer; - this.consumerPool = consumerPool; - } - - /** - * @return the Kafka Consumer for this - */ - public Consumer getConsumer() { - return consumer; - } - - /** - * Sets the poison flag for this consumer to true. - */ - public void poison() { - poisoned = true; - } - - /** - * @return true if this consumer has been poisoned, false otherwise - */ - public boolean isPoisoned() { - return poisoned; - } - - @Override - public void close() throws IOException { - consumerPool.returnConsumerResource(this); - } - -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessorLifecycleTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessorLifecycleTest.java deleted file mode 100644 index d09be60dd1..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessorLifecycleTest.java +++ /dev/null @@ -1,456 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; - -import java.io.Closeable; -import java.lang.reflect.Field; -import java.util.List; -import java.util.Random; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.AbstractSessionFactoryProcessor; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; -import org.junit.Ignore; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -public class AbstractKafkaProcessorLifecycleTest { - - private final static Random random = new Random(); - - @Test - public void validateBaseProperties() throws Exception { - TestRunner runner = TestRunners.newTestRunner(DummyProcessor.class); - runner.setProperty(AbstractKafkaProcessor.BOOTSTRAP_SERVERS, ""); - runner.setProperty(AbstractKafkaProcessor.TOPIC, "foo"); - runner.setProperty(ConsumeKafka.CLIENT_ID, "foo"); - - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "foo"); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("'bootstrap.servers' validated against 'foo' is invalid")); - } - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "foo:1234"); - - runner.removeProperty(ConsumeKafka.TOPIC); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("'topic' is invalid because topic is required")); - } - - runner.setProperty(ConsumeKafka.TOPIC, ""); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - - runner.setProperty(ConsumeKafka.TOPIC, " "); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - runner.setProperty(ConsumeKafka.TOPIC, "blah"); - - runner.removeProperty(ConsumeKafka.CLIENT_ID); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("invalid because client.id is required")); - } - - runner.setProperty(ConsumeKafka.CLIENT_ID, ""); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - - runner.setProperty(ConsumeKafka.CLIENT_ID, " "); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - runner.setProperty(ConsumeKafka.CLIENT_ID, "ghj"); - - runner.setProperty(PublishKafka.KERBEROS_PRINCIPLE, ""); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - runner.setProperty(PublishKafka.KERBEROS_PRINCIPLE, " "); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - } - - @Test - @Ignore // just for extra sanity check - public void validateConcurrencyWithRandomFailuresMultiple() throws Exception { - for (int i = 0; i < 100; i++) { - validateConcurrencyWithRandomFailures(); - } - } - - @Test - public void validateConcurrencyWithRandomFailures() throws Exception { - ExecutorService processingExecutor = Executors.newFixedThreadPool(32); - final AtomicInteger commitCounter = new AtomicInteger(); - final AtomicInteger rollbackCounter = new AtomicInteger(); - final AtomicInteger yieldCounter = new AtomicInteger(); - - final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class); - final ProcessSession session = mock(ProcessSession.class); - when(sessionFactory.createSession()).thenReturn(session); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - commitCounter.incrementAndGet(); - return null; - } - }).when(session).commit(); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - rollbackCounter.incrementAndGet(); - return null; - } - }).when(session).rollback(true); - - final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor()); - - int testCount = 1000; - final CountDownLatch latch = new CountDownLatch(testCount); - for (int i = 0; i < testCount; i++) { - processingExecutor.execute(new Runnable() { - @Override - public void run() { - ProcessContext context = mock(ProcessContext.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - yieldCounter.incrementAndGet(); - return null; - } - }).when(context).yield(); - if (random.nextInt(10) == 5) { - when(context.getName()).thenReturn("fail"); - } - try { - processor.onTrigger(context, sessionFactory); - } catch (Exception e) { - fail(); - } finally { - latch.countDown(); - } - } - }); - } - - assertTrue(latch.await(20000, TimeUnit.MILLISECONDS)); - processingExecutor.shutdown(); - - System.out.println("SUCCESS: " + processor.successfulTriggers); - System.out.println("FAILURE: " + processor.failedTriggers); - System.out.println("INIT: " + processor.resourceReinitialized); - System.out.println("YIELD CALLS: " + yieldCounter.get()); - System.out.println("COMMIT CALLS: " + commitCounter.get()); - System.out.println("ROLLBACK CALLS: " + rollbackCounter.get()); - System.out.println("Close CALLS: " + processor.closeCounter.get()); - - /* - * this has to be <= 1 since the last thread may come to finally block - * after acceptTask flag has been reset at which point the close will - * not be called (which is correct behavior since it will be invoked - * explicitly by the life-cycle operations of a running processor). - * - * You can actually observe the =1 behavior in the next test where it is - * always 0 close calls - */ - int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get(); - assertTrue(closeVsInitDiff <= 1); - - assertEquals(commitCounter.get(), processor.successfulTriggers.get()); - assertEquals(rollbackCounter.get(), processor.failedTriggers.get()); - - assertEquals(testCount, - processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get()); - } - - @Test - public void validateConcurrencyWithAllSuccesses() throws Exception { - ExecutorService processingExecutor = Executors.newFixedThreadPool(32); - final AtomicInteger commitCounter = new AtomicInteger(); - final AtomicInteger rollbackCounter = new AtomicInteger(); - final AtomicInteger yieldCounter = new AtomicInteger(); - - final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class); - final ProcessSession session = mock(ProcessSession.class); - when(sessionFactory.createSession()).thenReturn(session); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - commitCounter.incrementAndGet(); - return null; - } - }).when(session).commit(); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - rollbackCounter.incrementAndGet(); - return null; - } - }).when(session).rollback(true); - - final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor()); - - int testCount = 1000; - final CountDownLatch latch = new CountDownLatch(testCount); - for (int i = 0; i < testCount; i++) { - processingExecutor.execute(new Runnable() { - @Override - public void run() { - ProcessContext context = mock(ProcessContext.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - yieldCounter.incrementAndGet(); - return null; - } - }).when(context).yield(); - try { - processor.onTrigger(context, sessionFactory); - } catch (Exception e) { - fail(); - } finally { - latch.countDown(); - } - } - }); - } - - assertTrue(latch.await(30000, TimeUnit.MILLISECONDS)); - processingExecutor.shutdown(); - - System.out.println("SUCCESS: " + processor.successfulTriggers); - System.out.println("FAILURE: " + processor.failedTriggers); - System.out.println("INIT: " + processor.resourceReinitialized); - System.out.println("YIELD CALLS: " + yieldCounter.get()); - System.out.println("COMMIT CALLS: " + commitCounter.get()); - System.out.println("ROLLBACK CALLS: " + rollbackCounter.get()); - System.out.println("Close CALLS: " + processor.closeCounter.get()); - - /* - * unlike previous test this one will always be 1 since there are no - * failures - */ - int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get(); - assertEquals(1, closeVsInitDiff); - - assertEquals(commitCounter.get(), processor.successfulTriggers.get()); - assertEquals(rollbackCounter.get(), processor.failedTriggers.get()); - - assertEquals(testCount, - processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get()); - } - - @Test - public void validateConcurrencyWithAllFailures() throws Exception { - ExecutorService processingExecutor = Executors.newFixedThreadPool(32); - final AtomicInteger commitCounter = new AtomicInteger(); - final AtomicInteger rollbackCounter = new AtomicInteger(); - final AtomicInteger yieldCounter = new AtomicInteger(); - - final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class); - final ProcessSession session = mock(ProcessSession.class); - when(sessionFactory.createSession()).thenReturn(session); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - commitCounter.incrementAndGet(); - return null; - } - }).when(session).commit(); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - rollbackCounter.incrementAndGet(); - return null; - } - }).when(session).rollback(true); - - final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor()); - - int testCount = 1000; - final CountDownLatch latch = new CountDownLatch(testCount); - for (int i = 0; i < testCount; i++) { - processingExecutor.execute(new Runnable() { - @Override - public void run() { - ProcessContext context = mock(ProcessContext.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - yieldCounter.incrementAndGet(); - return null; - } - }).when(context).yield(); - when(context.getName()).thenReturn("fail"); - try { - processor.onTrigger(context, sessionFactory); - } catch (Exception e) { - fail(); - } finally { - latch.countDown(); - } - } - }); - } - - assertTrue(latch.await(20000, TimeUnit.MILLISECONDS)); - processingExecutor.shutdown(); - - System.out.println("SUCCESS: " + processor.successfulTriggers); - System.out.println("FAILURE: " + processor.failedTriggers); - System.out.println("INIT: " + processor.resourceReinitialized); - System.out.println("YIELD CALLS: " + yieldCounter.get()); - System.out.println("COMMIT CALLS: " + commitCounter.get()); - System.out.println("ROLLBACK CALLS: " + rollbackCounter.get()); - System.out.println("Close CALLS: " + processor.closeCounter.get()); - - /* - * unlike previous test this one will always be 0 since all triggers are - * failures - */ - int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get(); - assertEquals(0, closeVsInitDiff); - - assertEquals(commitCounter.get(), processor.successfulTriggers.get()); - assertEquals(rollbackCounter.get(), processor.failedTriggers.get()); - - assertEquals(testCount, - processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get()); - } - - /** - * - */ - public static class DummyProcessor extends AbstractKafkaProcessor { - @Override - protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) throws ProcessException { - return true; - } - - @Override - protected Closeable buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException { - return mock(Closeable.class); - } - - @Override - protected List getSupportedPropertyDescriptors() { - return SHARED_DESCRIPTORS; - } - } - - - public static class ConcurrencyValidatingProcessor extends AbstractKafkaProcessor { - final AtomicInteger failedTriggers = new AtomicInteger(); - final AtomicInteger successfulTriggers = new AtomicInteger(); - final AtomicInteger resourceReinitialized = new AtomicInteger(); - final AtomicInteger closeCounter = new AtomicInteger(); - - ConcurrencyValidatingProcessor() { - try { - Field loggerField = AbstractSessionFactoryProcessor.class.getDeclaredField("logger"); - loggerField.setAccessible(true); - loggerField.set(this, mock(ComponentLog.class)); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - @Override - @OnStopped - public void close() { - super.close(); - assertTrue(this.kafkaResource == null); - closeCounter.incrementAndGet(); - } - - @Override - protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) { - assertNotNull(this.kafkaResource); - if ("fail".equals(context.getName())) { - failedTriggers.incrementAndGet(); - throw new RuntimeException("Intentional"); - } - this.successfulTriggers.incrementAndGet(); - return true; - } - - @Override - protected Closeable buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException { - this.resourceReinitialized.incrementAndGet(); - return mock(Closeable.class); - } - } -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java deleted file mode 100644 index 8e17a21cd0..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.nio.charset.StandardCharsets; -import java.util.List; - -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; -import org.junit.Test; - -// The test is valid and should be ran when working on this module. @Ignore is -// to speed up the overall build -public class ConsumeKafkaTest { - - @Test - public void validateCustomSerilaizerDeserializerSettings() throws Exception { - ConsumeKafka consumeKafka = new ConsumeKafka(); - TestRunner runner = TestRunners.newTestRunner(consumeKafka); - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "okeydokey:1234"); - runner.setProperty(ConsumeKafka.TOPIC, "foo"); - runner.setProperty(ConsumeKafka.CLIENT_ID, "foo"); - runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); - runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); - runner.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); - runner.assertValid(); - runner.setProperty("key.deserializer", "Foo"); - runner.assertNotValid(); - } - - @Test - public void validatePropertiesValidation() throws Exception { - ConsumeKafka consumeKafka = new ConsumeKafka(); - TestRunner runner = TestRunners.newTestRunner(consumeKafka); - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "okeydokey:1234"); - runner.setProperty(ConsumeKafka.TOPIC, "foo"); - runner.setProperty(ConsumeKafka.CLIENT_ID, "foo"); - runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); - runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); - - runner.removeProperty(ConsumeKafka.GROUP_ID); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("invalid because group.id is required")); - } - - runner.setProperty(ConsumeKafka.GROUP_ID, ""); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - - runner.setProperty(ConsumeKafka.GROUP_ID, " "); - try { - runner.assertValid(); - fail(); - } catch (AssertionError e) { - assertTrue(e.getMessage().contains("must contain at least one character that is not white space")); - } - } - - /** - * Will set auto-offset to 'smallest' to ensure that all events (the once - * that were sent before and after consumer startup) are received. - */ - @Test - public void validateGetAllMessages() throws Exception { - String topicName = "validateGetAllMessages"; - - StubConsumeKafka consumeKafka = new StubConsumeKafka(); - - final TestRunner runner = TestRunners.newTestRunner(consumeKafka); - runner.setValidateExpressionUsage(false); - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); - runner.setProperty(ConsumeKafka.TOPIC, topicName); - runner.setProperty(ConsumeKafka.CLIENT_ID, "foo"); - runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); - runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); - runner.setProperty("check.connection", "false"); - - byte[][] values = new byte[][] { "Hello-1".getBytes(StandardCharsets.UTF_8), - "Hello-2".getBytes(StandardCharsets.UTF_8), "Hello-3".getBytes(StandardCharsets.UTF_8) }; - consumeKafka.setValues(values); - - runner.run(1, false); - - values = new byte[][] { "Hello-4".getBytes(StandardCharsets.UTF_8), "Hello-5".getBytes(StandardCharsets.UTF_8), - "Hello-6".getBytes(StandardCharsets.UTF_8) }; - consumeKafka.setValues(values); - - runner.run(1, false); - - final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); - - assertEquals(6, flowFiles.size()); - // spot check - MockFlowFile flowFile = flowFiles.get(0); - String event = new String(flowFile.toByteArray()); - assertEquals("Hello-1", event); - - flowFile = flowFiles.get(1); - event = new String(flowFile.toByteArray()); - assertEquals("Hello-2", event); - - flowFile = flowFiles.get(5); - event = new String(flowFile.toByteArray()); - assertEquals("Hello-6", event); - - consumeKafka.close(); - } - - @Test - public void validateGetAllMessagesWithProvidedDemarcator() throws Exception { - String topicName = "validateGetAllMessagesWithProvidedDemarcator"; - - StubConsumeKafka consumeKafka = new StubConsumeKafka(); - - final TestRunner runner = TestRunners.newTestRunner(consumeKafka); - runner.setValidateExpressionUsage(false); - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); - runner.setProperty(ConsumeKafka.TOPIC, topicName); - runner.setProperty(ConsumeKafka.CLIENT_ID, "foo"); - runner.setProperty(ConsumeKafka.GROUP_ID, "foo"); - runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST); - runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah"); - runner.setProperty("check.connection", "false"); - - byte[][] values = new byte[][] { "Hello-1".getBytes(StandardCharsets.UTF_8), - "Hi-2".getBytes(StandardCharsets.UTF_8) }; - consumeKafka.setValues(values); - - runner.run(1, false); - List flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); - assertEquals(1, flowFiles.size()); - - values = new byte[][] { "Здравствуйте-3".getBytes(StandardCharsets.UTF_8), - "こんにちは-4".getBytes(StandardCharsets.UTF_8), "Hello-5".getBytes(StandardCharsets.UTF_8) }; - consumeKafka.setValues(values); - - runner.run(1, false); - - flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS); - - assertEquals(2, flowFiles.size()); - MockFlowFile flowFile = flowFiles.get(0); - String[] events = new String(flowFile.toByteArray(), StandardCharsets.UTF_8).split("blah"); - assertEquals("0", flowFile.getAttribute("kafka.partition")); - assertEquals("0", flowFile.getAttribute("kafka.offset")); - assertEquals("validateGetAllMessagesWithProvidedDemarcator", flowFile.getAttribute("kafka.topic")); - - assertEquals(2, events.length); - - flowFile = flowFiles.get(1); - events = new String(flowFile.toByteArray(), StandardCharsets.UTF_8).split("blah"); - - assertEquals(3, events.length); - // spot check - assertEquals("Здравствуйте-3", events[0]); - assertEquals("こんにちは-4", events[1]); - assertEquals("Hello-5", events[2]); - - consumeKafka.close(); - } -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubConsumeKafka.java deleted file mode 100644 index 0f0b23fd8d..0000000000 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubConsumeKafka.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.kafka.pubsub; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.TopicPartition; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -public class StubConsumeKafka extends ConsumeKafka { - - private byte[][] values; - - public void setValues(byte[][] values) { - this.values = values; - } - - - @SuppressWarnings("unchecked") - @Override - protected Consumer buildKafkaResource(ProcessContext context, ProcessSession session) { - Consumer consumer = super.buildKafkaResource(context, session); - consumer = mock(Consumer.class); - String topicName = context.getProperty(TOPIC).evaluateAttributeExpressions().getValue(); - - when(consumer.poll(Mockito.anyLong())).thenAnswer(new Answer>() { - @Override - public ConsumerRecords answer(InvocationOnMock invocation) throws Throwable { - List> records = new ArrayList<>(); - for (int i = 0; i < StubConsumeKafka.this.values.length; i++) { - byte[] value = StubConsumeKafka.this.values[i]; - ConsumerRecord record = new ConsumerRecord<>(topicName, 0, 0, null, value); - records.add(record); - } - TopicPartition partition = new TopicPartition(topicName, 0); - Map>> m = new LinkedHashMap<>(); - m.put(partition, records); - return new ConsumerRecords<>(m); - } - }); - - return consumer; - } -} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml index fb5a64f276..1a258f9f9f 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml @@ -22,21 +22,28 @@ nifi-kafka-bundle pom - nifi-kafka-processors - nifi-kafka-pubsub-processors - nifi-kafka-nar - nifi-kafka-pubsub-nar + nifi-kafka-0-8-processors + nifi-kafka-0-9-processors + nifi-kafka-0-10-processors + nifi-kafka-0-8-nar + nifi-kafka-0-9-nar + nifi-kafka-0-10-nar org.apache.nifi - nifi-kafka-processors + nifi-kafka-0-8-processors 1.0.0-SNAPSHOT org.apache.nifi - nifi-kafka-pubsub-processors + nifi-kafka-0-10-processors + 1.0.0-SNAPSHOT + + + org.apache.nifi + nifi-kafka-0-9-processors 1.0.0-SNAPSHOT diff --git a/pom.xml b/pom.xml index 564f96e706..126eb145bc 100644 --- a/pom.xml +++ b/pom.xml @@ -996,13 +996,19 @@ language governing permissions and limitations under the License. --> org.apache.nifi - nifi-kafka-nar + nifi-kafka-0-8-nar 1.0.0-SNAPSHOT nar org.apache.nifi - nifi-kafka-pubsub-nar + nifi-kafka-0-9-nar + 1.0.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-kafka-0-10-nar 1.0.0-SNAPSHOT nar