diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index b1ab989e67..ddcd6e8a6c 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -196,6 +196,11 @@ language governing permissions and limitations under the License. --> nifi-kafka-0-11-nar nar + + org.apache.nifi + nifi-kafka-1-0-nar + nar + org.apache.nifi nifi-confluent-platform-nar diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_10/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_11/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_11/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_10/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_11/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_10/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_11/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka_0_10/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka_0_11/additionalDetails.html similarity index 100% rename from nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka_0_10/additionalDetails.html rename to nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka_0_11/additionalDetails.html diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml new file mode 100644 index 0000000000..f1806a165c --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/pom.xml @@ -0,0 +1,40 @@ + + + 4.0.0 + + org.apache.nifi + nifi-kafka-bundle + 1.5.0-SNAPSHOT + + nifi-kafka-1-0-nar + nar + NiFi NAR for interacting with Apache Kafka 1.0 + + true + true + + + + org.apache.nifi + nifi-kafka-1-0-processors + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 0000000000..43a2a3b5de --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,233 @@ + + 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 'Bouncy Castle JDK 1.5' + under an MIT style license. + + Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org) + + 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. + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..03af5fd517 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,83 @@ +nifi-kafka-nar +Copyright 2014-2017 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-2017 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + (ASLv2) Apache Commons IO + The following NOTICE information applies: + Apache Commons IO + Copyright 2002-2016 The Apache Software Foundation + + (ASLv2) Apache Commons Codec + The following NOTICE information applies: + Apache Commons Codec + Copyright 2002-2014 The Apache Software Foundation + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + (ASLv2) Apache Kafka + The following NOTICE information applies: + Apache Kafka + Copyright 2012 The Apache Software Foundation. + + (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) Jackson JSON processor + The following NOTICE information applies: + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml new file mode 100644 index 0000000000..0f51298b3c --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/pom.xml @@ -0,0 +1,92 @@ + + + + org.apache.nifi + nifi-kafka-bundle + 1.5.0-SNAPSHOT + + 4.0.0 + nifi-kafka-1-0-processors + jar + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-record-serialization-service-api + + + org.apache.nifi + nifi-record + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-utils + + + org.apache.nifi + nifi-ssl-context-service-api + + + org.apache.kafka + kafka-clients + ${kafka1.0.version} + + + org.apache.kafka + kafka_2.11 + ${kafka1.0.version} + 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-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java new file mode 100644 index 0000000000..f0c1bd0c45 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_1_0.java @@ -0,0 +1,395 @@ +/* + * 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.Charset; +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.Set; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.WakeupException; +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.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +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.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 org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriterFactory; + +@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 1.0 Consumer API. " + + "The complementary NiFi processor for sending messages is PublishKafkaRecord_1_0. Please note that, at this time, the Processor assumes that " + + "all records that are retrieved from a given partition have the same schema. If any of the Kafka messages are pulled but cannot be parsed or written with the " + + "configured Record Reader or Record Writer, the contents of the message will be written to a separate FlowFile, and that FlowFile will be transferred to the " + + "'parse.failure' relationship. Otherwise, each FlowFile is sent to the 'success' relationship and may contain many individual messages within the single FlowFile. " + + "A 'record.count' attribute is added to indicate how many messages are contained in the FlowFile. No two Kafka messages will be placed into the same FlowFile if they " + + "have different schemas, or if they have different values for a message header that is included by the property.") +@Tags({"Kafka", "Get", "Record", "csv", "avro", "json", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "1.0"}) +@WritesAttributes({ + @WritesAttribute(attribute = "record.count", description = "The number of records received"), + @WritesAttribute(attribute = "mime.type", description = "The MIME Type that is provided by the configured Record Writer"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the records are from"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic records are 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. ") +@SeeAlso({ConsumeKafka_1_0.class, PublishKafka_1_0.class, PublishKafkaRecord_1_0.class}) +public class ConsumeKafkaRecord_1_0 extends AbstractProcessor { + + 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 AllowableValue TOPIC_NAME = new AllowableValue("names", "names", "Topic is a full topic name or comma separated list of names"); + static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", "pattern", "Topic is a regex using the Java Pattern syntax"); + + 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 separated.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder() + .name("topic_type") + .displayName("Topic Name Format") + .description("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression") + .required(true) + .allowableValues(TOPIC_NAME, TOPIC_PATTERN) + .defaultValue(TOPIC_NAME.getValue()) + .build(); + + static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder() + .name("record-reader") + .displayName("Record Reader") + .description("The Record Reader to use for incoming FlowFiles") + .identifiesControllerService(RecordReaderFactory.class) + .expressionLanguageSupported(false) + .required(true) + .build(); + + static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() + .name("record-writer") + .displayName("Record Writer") + .description("The Record Writer to use in order to serialize the data before sending to Kafka") + .identifiesControllerService(RecordSetWriterFactory.class) + .expressionLanguageSupported(false) + .required(true) + .build(); + + static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder() + .name("group.id") + .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("auto.offset.reset") + .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 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 PropertyDescriptor MAX_UNCOMMITTED_TIME = new PropertyDescriptor.Builder() + .name("max-uncommit-offset-wait") + .displayName("Max Uncommitted Time") + .description("Specifies the maximum amount of time allowed to pass before offsets must be committed. " + + "This value impacts how often offsets will be committed. Committing offsets less often increases " + + "throughput but also increases the window of potential data duplication in the event of a rebalance " + + "or JVM restart between commits. This value is also related to maximum poll records and the use " + + "of a message demarcator. When using a message demarcator we can have far more uncommitted messages " + + "than when we're not as there is much less for us to keep track of in memory.") + .required(false) + .defaultValue("1 secs") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + static final PropertyDescriptor HONOR_TRANSACTIONS = new PropertyDescriptor.Builder() + .name("honor-transactions") + .displayName("Honor Transactions") + .description("Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If " + + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait " + + "for the producer to finish its entire transaction instead of pulling as the messages become available.") + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); + static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder() + .name("message-header-encoding") + .displayName("Message Header Encoding") + .description("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. " + + "This property indicates the Character Encoding to use for deserializing the headers.") + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .required(false) + .build(); + static final PropertyDescriptor HEADER_NAME_REGEX = new PropertyDescriptor.Builder() + .name("header-name-regex") + .displayName("Headers to Add as Attributes (Regex)") + .description("A Regular Expression that is matched against all message headers. " + + "Any message header whose name matches the regex will be added to the FlowFile as an Attribute. " + + "If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that header is selected by " + + "the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a regex like " + + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling " + + "the messages together efficiently.") + .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) + .expressionLanguageSupported(false) + .required(false) + .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 Relationship REL_PARSE_FAILURE = new Relationship.Builder() + .name("parse.failure") + .description("If a message from Kafka cannot be parsed using the configured Record Reader, the contents of the " + + "message will be routed to this Relationship as its own individual FlowFile.") + .build(); + + static final List DESCRIPTORS; + static final Set RELATIONSHIPS; + + private volatile ConsumerPool consumerPool = null; + private final Set activeLeases = Collections.synchronizedSet(new HashSet<>()); + + static { + List descriptors = new ArrayList<>(); + descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS); + descriptors.add(TOPICS); + descriptors.add(TOPIC_TYPE); + descriptors.add(RECORD_READER); + descriptors.add(RECORD_WRITER); + descriptors.add(HONOR_TRANSACTIONS); + descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL); + descriptors.add(KafkaProcessorUtils.KERBEROS_PRINCIPLE); + descriptors.add(KafkaProcessorUtils.USER_PRINCIPAL); + descriptors.add(KafkaProcessorUtils.USER_KEYTAB); + descriptors.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE); + descriptors.add(GROUP_ID); + descriptors.add(AUTO_OFFSET_RESET); + descriptors.add(MESSAGE_HEADER_ENCODING); + descriptors.add(HEADER_NAME_REGEX); + descriptors.add(MAX_POLL_RECORDS); + descriptors.add(MAX_UNCOMMITTED_TIME); + DESCRIPTORS = Collections.unmodifiableList(descriptors); + + final Set rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_PARSE_FAILURE); + RELATIONSHIPS = Collections.unmodifiableSet(rels); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @OnStopped + public void close() { + 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; + } + + return consumerPool = createConsumerPool(context, getLogger()); + } + + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + final int maxLeases = context.getMaxConcurrentTasks(); + final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS); + + final Map props = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props); + 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()); + final String topicListing = context.getProperty(ConsumeKafkaRecord_1_0.TOPICS).evaluateAttributeExpressions().getValue(); + final String topicType = context.getProperty(ConsumeKafkaRecord_1_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue(); + final List topics = new ArrayList<>(); + final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(); + final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + + final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); + final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class); + final boolean honorTransactions = context.getProperty(HONOR_TRANSACTIONS).asBoolean(); + + final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue(); + final Charset charset = Charset.forName(charsetName); + + final String headerNameRegex = context.getProperty(HEADER_NAME_REGEX).getValue(); + final Pattern headerNamePattern = headerNameRegex == null ? null : Pattern.compile(headerNameRegex); + + if (topicType.equals(TOPIC_NAME.getValue())) { + for (final String topic : topicListing.split(",", 100)) { + final String trimmedName = topic.trim(); + if (!trimmedName.isEmpty()) { + topics.add(trimmedName); + } + } + + return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topics, maxUncommittedTime, securityProtocol, + bootstrapServers, log, honorTransactions, charset, headerNamePattern); + } else if (topicType.equals(TOPIC_PATTERN.getValue())) { + final Pattern topicPattern = Pattern.compile(topicListing.trim()); + return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topicPattern, maxUncommittedTime, securityProtocol, + bootstrapServers, log, honorTransactions, charset, headerNamePattern); + } else { + getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType}); + return null; + } + } + + @OnUnscheduled + public void interruptActiveThreads() { + // There are known issues with the Kafka client library that result in the client code hanging + // indefinitely when unable to communicate with the broker. In order to address this, we will wait + // up to 30 seconds for the Threads to finish and then will call Consumer.wakeup() to trigger the + // thread to wakeup when it is blocked, waiting on a response. + final long nanosToWait = TimeUnit.SECONDS.toNanos(5L); + final long start = System.nanoTime(); + while (System.nanoTime() - start < nanosToWait && !activeLeases.isEmpty()) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + + if (!activeLeases.isEmpty()) { + int count = 0; + for (final ConsumerLease lease : activeLeases) { + getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease}); + lease.wakeup(); + count++; + } + + getLogger().info("Woke up {} consumers", new Object[] {count}); + } + + activeLeases.clear(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final ConsumerPool pool = getConsumerPool(context); + if (pool == null) { + context.yield(); + return; + } + + try (final ConsumerLease lease = pool.obtainConsumer(session, context)) { + if (lease == null) { + context.yield(); + return; + } + + activeLeases.add(lease); + try { + while (this.isScheduled() && lease.continuePolling()) { + lease.poll(); + } + if (this.isScheduled() && !lease.commit()) { + context.yield(); + } + } catch (final WakeupException we) { + getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. " + + "Will roll back session and discard any partially received data.", new Object[] {lease}); + } catch (final KafkaException kex) { + getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}", + new Object[]{lease, kex}, kex); + } catch (final Throwable t) { + getLogger().error("Exception while processing data from kafka so will close the lease {} due to {}", + new Object[]{lease, t}, t); + } finally { + activeLeases.remove(lease); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java new file mode 100644 index 0000000000..fdc2cb33a6 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_1_0.java @@ -0,0 +1,386 @@ +/* + * 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.Charset; +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.Set; +import java.util.regex.Pattern; +import java.util.concurrent.TimeUnit; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.WakeupException; +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.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +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.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.HEX_ENCODING; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; + +@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 1.0 Consumer API. " + + "The complementary NiFi processor for sending messages is PublishKafka_1_0.") +@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "1.0"}) +@WritesAttributes({ + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"), + @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY, description = "The key of message if present and if single message. " + + "How the key is encoded depends on the value of the 'Key Attribute Encoding' property."), + @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_1_0 extends AbstractProcessor { + + 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 AllowableValue TOPIC_NAME = new AllowableValue("names", "names", "Topic is a full topic name or comma separated list of names"); + + static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", "pattern", "Topic is a regex using the Java Pattern syntax"); + + 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 separated.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor TOPIC_TYPE = new PropertyDescriptor.Builder() + .name("topic_type") + .displayName("Topic Name Format") + .description("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression") + .required(true) + .allowableValues(TOPIC_NAME, TOPIC_PATTERN) + .defaultValue(TOPIC_NAME.getValue()) + .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 KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder() + .name("key-attribute-encoding") + .displayName("Key Attribute Encoding") + .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.") + .required(true) + .defaultValue(UTF8_ENCODING.getValue()) + .allowableValues(UTF8_ENCODING, HEX_ENCODING) + .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 HEADER_NAME_REGEX = new PropertyDescriptor.Builder() + .name("header-name-regex") + .displayName("Headers to Add as Attributes (Regex)") + .description("A Regular Expression that is matched against all message headers. " + + "Any message header whose name matches the regex will be added to the FlowFile as an Attribute. " + + "If not specified, no Header values will be added as FlowFile attributes. If two messages have a different value for the same header and that header is selected by " + + "the provided regex, then those two messages must be added to different FlowFiles. As a result, users should be cautious about using a regex like " + + "\".*\" if messages are expected to have header values that are unique per message, such as an identifier or timestamp, because it will prevent NiFi from bundling " + + "the messages together efficiently.") + .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) + .expressionLanguageSupported(false) + .required(false) + .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 PropertyDescriptor MAX_UNCOMMITTED_TIME = new PropertyDescriptor.Builder() + .name("max-uncommit-offset-wait") + .displayName("Max Uncommitted Time") + .description("Specifies the maximum amount of time allowed to pass before offsets must be committed. " + + "This value impacts how often offsets will be committed. Committing offsets less often increases " + + "throughput but also increases the window of potential data duplication in the event of a rebalance " + + "or JVM restart between commits. This value is also related to maximum poll records and the use " + + "of a message demarcator. When using a message demarcator we can have far more uncommitted messages " + + "than when we're not as there is much less for us to keep track of in memory.") + .required(false) + .defaultValue("1 secs") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + + static final PropertyDescriptor HONOR_TRANSACTIONS = new PropertyDescriptor.Builder() + .name("honor-transactions") + .displayName("Honor Transactions") + .description("Specifies whether or not NiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of " + + "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. If " + + "this value is true, NiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer must wait " + + "for the producer to finish its entire transaction instead of pulling as the messages become available.") + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); + static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder() + .name("message-header-encoding") + .displayName("Message Header Encoding") + .description("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. " + + "This property indicates the Character Encoding to use for deserializing the headers.") + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .required(false) + .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 ConsumerPool consumerPool = null; + private final Set activeLeases = Collections.synchronizedSet(new HashSet<>()); + + static { + List descriptors = new ArrayList<>(); + descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + descriptors.add(TOPICS); + descriptors.add(TOPIC_TYPE); + descriptors.add(HONOR_TRANSACTIONS); + descriptors.add(GROUP_ID); + descriptors.add(AUTO_OFFSET_RESET); + descriptors.add(KEY_ATTRIBUTE_ENCODING); + descriptors.add(MESSAGE_DEMARCATOR); + descriptors.add(MESSAGE_HEADER_ENCODING); + descriptors.add(HEADER_NAME_REGEX); + descriptors.add(MAX_POLL_RECORDS); + descriptors.add(MAX_UNCOMMITTED_TIME); + DESCRIPTORS = Collections.unmodifiableList(descriptors); + RELATIONSHIPS = Collections.singleton(REL_SUCCESS); + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @OnStopped + public void close() { + 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; + } + + return consumerPool = createConsumerPool(context, getLogger()); + } + + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + final int maxLeases = context.getMaxConcurrentTasks(); + final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS); + final byte[] demarcator = context.getProperty(ConsumeKafka_1_0.MESSAGE_DEMARCATOR).isSet() + ? context.getProperty(ConsumeKafka_1_0.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) + : null; + final Map props = new HashMap<>(); + KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props); + 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()); + + final String topicListing = context.getProperty(ConsumeKafka_1_0.TOPICS).evaluateAttributeExpressions().getValue(); + final String topicType = context.getProperty(ConsumeKafka_1_0.TOPIC_TYPE).evaluateAttributeExpressions().getValue(); + final List topics = new ArrayList<>(); + final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue(); + final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(); + final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + final boolean honorTransactions = context.getProperty(HONOR_TRANSACTIONS).asBoolean(); + + final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue(); + final Charset charset = Charset.forName(charsetName); + + final String headerNameRegex = context.getProperty(HEADER_NAME_REGEX).getValue(); + final Pattern headerNamePattern = headerNameRegex == null ? null : Pattern.compile(headerNameRegex); + + if (topicType.equals(TOPIC_NAME.getValue())) { + for (final String topic : topicListing.split(",", 100)) { + final String trimmedName = topic.trim(); + if (!trimmedName.isEmpty()) { + topics.add(trimmedName); + } + } + + return new ConsumerPool(maxLeases, demarcator, props, topics, maxUncommittedTime, keyEncoding, securityProtocol, + bootstrapServers, log, honorTransactions, charset, headerNamePattern); + } else if (topicType.equals(TOPIC_PATTERN.getValue())) { + final Pattern topicPattern = Pattern.compile(topicListing.trim()); + return new ConsumerPool(maxLeases, demarcator, props, topicPattern, maxUncommittedTime, keyEncoding, securityProtocol, + bootstrapServers, log, honorTransactions, charset, headerNamePattern); + } else { + getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType}); + return null; + } + } + + @OnUnscheduled + public void interruptActiveThreads() { + // There are known issues with the Kafka client library that result in the client code hanging + // indefinitely when unable to communicate with the broker. In order to address this, we will wait + // up to 30 seconds for the Threads to finish and then will call Consumer.wakeup() to trigger the + // thread to wakeup when it is blocked, waiting on a response. + final long nanosToWait = TimeUnit.SECONDS.toNanos(5L); + final long start = System.nanoTime(); + while (System.nanoTime() - start < nanosToWait && !activeLeases.isEmpty()) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + + if (!activeLeases.isEmpty()) { + int count = 0; + for (final ConsumerLease lease : activeLeases) { + getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease}); + lease.wakeup(); + count++; + } + + getLogger().info("Woke up {} consumers", new Object[] {count}); + } + + activeLeases.clear(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final ConsumerPool pool = getConsumerPool(context); + if (pool == null) { + context.yield(); + return; + } + + try (final ConsumerLease lease = pool.obtainConsumer(session, context)) { + if (lease == null) { + context.yield(); + return; + } + + activeLeases.add(lease); + try { + while (this.isScheduled() && lease.continuePolling()) { + lease.poll(); + } + if (this.isScheduled() && !lease.commit()) { + context.yield(); + } + } catch (final WakeupException we) { + getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. " + + "Will roll back session and discard any partially received data.", new Object[] {lease}); + } catch (final KafkaException kex) { + getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}", + new Object[]{lease, kex}, kex); + } catch (final Throwable t) { + getLogger().error("Exception while processing data from kafka so will close the lease {} due to {}", + new Object[]{lease, t}, t); + } finally { + activeLeases.remove(lease); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java new file mode 100644 index 0000000000..a2a449c09c --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -0,0 +1,701 @@ +/* + * 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.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_PARSE_FAILURE; +import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_SUCCESS; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; + +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import javax.xml.bind.DatatypeConverter; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +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.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; + +/** + * 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 abstract class ConsumerLease implements Closeable, ConsumerRebalanceListener { + + private final long maxWaitMillis; + private final Consumer kafkaConsumer; + private final ComponentLog logger; + private final byte[] demarcatorBytes; + private final String keyEncoding; + private final String securityProtocol; + private final String bootstrapServers; + private final RecordSetWriterFactory writerFactory; + private final RecordReaderFactory readerFactory; + private final Charset headerCharacterSet; + private final Pattern headerNamePattern; + private boolean poisoned = false; + //used for tracking demarcated flowfiles to their TopicPartition so we can append + //to them on subsequent poll calls + private final Map bundleMap = new HashMap<>(); + private final Map uncommittedOffsetsMap = new HashMap<>(); + private long leaseStartNanos = -1; + private boolean lastPollEmpty = false; + private int totalMessages = 0; + + ConsumerLease( + final long maxWaitMillis, + final Consumer kafkaConsumer, + final byte[] demarcatorBytes, + final String keyEncoding, + final String securityProtocol, + final String bootstrapServers, + final RecordReaderFactory readerFactory, + final RecordSetWriterFactory writerFactory, + final ComponentLog logger, + final Charset headerCharacterSet, + final Pattern headerNamePattern) { + this.maxWaitMillis = maxWaitMillis; + this.kafkaConsumer = kafkaConsumer; + this.demarcatorBytes = demarcatorBytes; + this.keyEncoding = keyEncoding; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.readerFactory = readerFactory; + this.writerFactory = writerFactory; + this.logger = logger; + this.headerCharacterSet = headerCharacterSet; + this.headerNamePattern = headerNamePattern; + } + + /** + * clears out internal state elements excluding session and consumer as + * those are managed by the pool itself + */ + private void resetInternalState() { + bundleMap.clear(); + uncommittedOffsetsMap.clear(); + leaseStartNanos = -1; + lastPollEmpty = false; + totalMessages = 0; + } + + /** + * Kafka will call this method whenever it is about to rebalance the + * consumers for the given partitions. We'll simply take this to mean that + * we need to quickly commit what we've got and will return the consumer to + * the pool. This method will be called during the poll() method call of + * this class and will be called by the same thread calling poll according + * to the Kafka API docs. After this method executes the session and kafka + * offsets are committed and this lease is closed. + * + * @param partitions partitions being reassigned + */ + @Override + public void onPartitionsRevoked(final Collection partitions) { + logger.debug("Rebalance Alert: Paritions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer}); + //force a commit here. Can reuse the session and consumer after this but must commit now to avoid duplicates if kafka reassigns partition + commit(); + } + + /** + * This will be called by Kafka when the rebalance has completed. We don't + * need to do anything with this information other than optionally log it as + * by this point we've committed what we've got and moved on. + * + * @param partitions topic partition set being reassigned + */ + @Override + public void onPartitionsAssigned(final Collection partitions) { + logger.debug("Rebalance Alert: Paritions '{}' assigned for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer}); + } + + /** + * Executes a poll on the underlying Kafka Consumer and creates any new + * flowfiles necessary or appends to existing ones if in demarcation mode. + */ + void poll() { + /** + * Implementation note: + * Even if ConsumeKafka is not scheduled to poll due to downstream connection back-pressure is engaged, + * for longer than session.timeout.ms (defaults to 10 sec), Kafka consumer sends heartbeat from background thread. + * If this situation lasts longer than max.poll.interval.ms (defaults to 5 min), Kafka consumer sends + * Leave Group request to Group Coordinator. When ConsumeKafka processor is scheduled again, Kafka client checks + * if this client instance is still a part of consumer group. If not, it rejoins before polling messages. + * This behavior has been fixed via Kafka KIP-62 and available from Kafka client 0.10.1.0. + */ + try { + final ConsumerRecords records = kafkaConsumer.poll(10); + lastPollEmpty = records.count() == 0; + processRecords(records); + } catch (final ProcessException pe) { + throw pe; + } catch (final Throwable t) { + this.poison(); + throw t; + } + } + + /** + * 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. + * + * if false then we didn't do anything and should probably yield if true + * then we committed new data + * + */ + boolean commit() { + if (uncommittedOffsetsMap.isEmpty()) { + resetInternalState(); + return false; + } + try { + /** + * Committing the nifi session then the offsets means we have an at + * least once guarantee here. If we reversed the order we'd have at + * most once. + */ + final Collection bundledFlowFiles = getBundles(); + if (!bundledFlowFiles.isEmpty()) { + getProcessSession().transfer(bundledFlowFiles, REL_SUCCESS); + } + getProcessSession().commit(); + + final Map offsetsMap = uncommittedOffsetsMap; + kafkaConsumer.commitSync(offsetsMap); + resetInternalState(); + return true; + } catch (final IOException ioe) { + poison(); + logger.error("Failed to finish writing out FlowFile bundle", ioe); + throw new ProcessException(ioe); + } catch (final KafkaException kex) { + poison(); + logger.warn("Duplicates are likely as we were able to commit the process" + + " session but received an exception from Kafka while committing" + + " offsets."); + throw kex; + } catch (final Throwable t) { + poison(); + throw t; + } + } + + /** + * Indicates whether we should continue polling for data. If we are not + * writing data with a demarcator then we're writing individual flow files + * per kafka message therefore we must be very mindful of memory usage for + * the flow file objects (not their content) being held in memory. The + * content of kafka messages will be written to the content repository + * immediately upon each poll call but we must still be mindful of how much + * memory can be used in each poll call. We will indicate that we should + * stop polling our last poll call produced no new results or if we've + * polling and processing data longer than the specified maximum polling + * time or if we have reached out specified max flow file limit or if a + * rebalance has been initiated for one of the partitions we're watching; + * otherwise true. + * + * @return true if should keep polling; false otherwise + */ + boolean continuePolling() { + //stop if the last poll produced new no data + if (lastPollEmpty) { + return false; + } + + //stop if we've gone past our desired max uncommitted wait time + if (leaseStartNanos < 0) { + leaseStartNanos = System.nanoTime(); + } + final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos); + if (durationMillis > maxWaitMillis) { + return false; + } + + //stop if we've generated enough flowfiles that we need to be concerned about memory usage for the objects + if (bundleMap.size() > 200) { //a magic number - the number of simultaneous bundles to track + return false; + } else { + return totalMessages < 1000;//admittedlly a magic number - good candidate for processor property + } + } + + /** + * Indicates that the underlying session and consumer should be immediately + * considered invalid. Once closed the session will be rolled back and the + * pool should destroy the underlying consumer. This is useful if due to + * external reasons, such as the processor no longer being scheduled, this + * lease should be terminated immediately. + */ + private void poison() { + poisoned = true; + } + + /** + * @return true if this lease has been poisoned; false otherwise + */ + boolean isPoisoned() { + return poisoned; + } + + /** + * Trigger the consumer's {@link KafkaConsumer#wakeup() wakeup()} method. + */ + public void wakeup() { + kafkaConsumer.wakeup(); + } + + /** + * Abstract method that is intended to be extended by the pool that created + * this ConsumerLease object. It should ensure that the session given to + * create this session is rolled back and that the underlying kafka consumer + * is either returned to the pool for continued use or destroyed if this + * lease has been poisoned. It can only be called once. Calling it more than + * once can result in undefined and non threadsafe behavior. + */ + @Override + public void close() { + resetInternalState(); + } + + public abstract ProcessSession getProcessSession(); + + public abstract void yield(); + + private void processRecords(final ConsumerRecords records) { + records.partitions().stream().forEach(partition -> { + List> messages = records.records(partition); + if (!messages.isEmpty()) { + //update maximum offset map for this topic partition + long maxOffset = messages.stream() + .mapToLong(record -> record.offset()) + .max() + .getAsLong(); + + //write records to content repository and session + if (demarcatorBytes != null) { + writeDemarcatedData(getProcessSession(), messages, partition); + } else if (readerFactory != null && writerFactory != null) { + writeRecordData(getProcessSession(), messages, partition); + } else { + messages.stream().forEach(message -> { + writeData(getProcessSession(), message, partition); + }); + } + + totalMessages += messages.size(); + uncommittedOffsetsMap.put(partition, new OffsetAndMetadata(maxOffset + 1L)); + } + }); + } + + private static String encodeKafkaKey(final byte[] key, final String encoding) { + if (key == null) { + return null; + } + + if (HEX_ENCODING.getValue().equals(encoding)) { + return DatatypeConverter.printHexBinary(key); + } else if (UTF8_ENCODING.getValue().equals(encoding)) { + return new String(key, StandardCharsets.UTF_8); + } else { + return null; // won't happen because it is guaranteed by the Allowable Values + } + } + + private Collection getBundles() throws IOException { + final List flowFiles = new ArrayList<>(); + for (final BundleTracker tracker : bundleMap.values()) { + final boolean includeBundle = processBundle(tracker); + if (includeBundle) { + flowFiles.add(tracker.flowFile); + } + } + return flowFiles; + } + + private boolean processBundle(final BundleTracker bundle) throws IOException { + final RecordSetWriter writer = bundle.recordWriter; + if (writer != null) { + final WriteResult writeResult; + + try { + writeResult = writer.finishRecordSet(); + } finally { + writer.close(); + } + + if (writeResult.getRecordCount() == 0) { + getProcessSession().remove(bundle.flowFile); + return false; + } + + final Map attributes = new HashMap<>(); + attributes.putAll(writeResult.getAttributes()); + attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); + + bundle.flowFile = getProcessSession().putAllAttributes(bundle.flowFile, attributes); + } + + populateAttributes(bundle); + return true; + } + + private void writeData(final ProcessSession session, ConsumerRecord record, final TopicPartition topicPartition) { + FlowFile flowFile = session.create(); + final BundleTracker tracker = new BundleTracker(record, topicPartition, keyEncoding); + tracker.incrementRecordCount(1); + final byte[] value = record.value(); + if (value != null) { + flowFile = session.write(flowFile, out -> { + out.write(value); + }); + } + flowFile = session.putAllAttributes(flowFile, getAttributes(record)); + tracker.updateFlowFile(flowFile); + populateAttributes(tracker); + session.transfer(tracker.flowFile, REL_SUCCESS); + } + + private void writeDemarcatedData(final ProcessSession session, final List> records, final TopicPartition topicPartition) { + // Group the Records by their BundleInformation + final Map>> map = records.stream() + .collect(Collectors.groupingBy(rec -> new BundleInformation(topicPartition, null, getAttributes(rec)))); + + for (final Map.Entry>> entry : map.entrySet()) { + final BundleInformation bundleInfo = entry.getKey(); + final List> recordList = entry.getValue(); + + final boolean demarcateFirstRecord; + + BundleTracker tracker = bundleMap.get(bundleInfo); + + FlowFile flowFile; + if (tracker == null) { + tracker = new BundleTracker(recordList.get(0), topicPartition, keyEncoding); + flowFile = session.create(); + flowFile = session.putAllAttributes(flowFile, bundleInfo.attributes); + tracker.updateFlowFile(flowFile); + demarcateFirstRecord = false; //have not yet written records for this topic/partition in this lease + } else { + demarcateFirstRecord = true; //have already been writing records for this topic/partition in this lease + } + flowFile = tracker.flowFile; + + tracker.incrementRecordCount(recordList.size()); + flowFile = session.append(flowFile, out -> { + boolean useDemarcator = demarcateFirstRecord; + for (final ConsumerRecord record : recordList) { + if (useDemarcator) { + out.write(demarcatorBytes); + } + final byte[] value = record.value(); + if (value != null) { + out.write(record.value()); + } + useDemarcator = true; + } + }); + + tracker.updateFlowFile(flowFile); + bundleMap.put(bundleInfo, tracker); + } + } + + private void handleParseFailure(final ConsumerRecord consumerRecord, final ProcessSession session, final Exception cause) { + handleParseFailure(consumerRecord, session, cause, "Failed to parse message from Kafka using the configured Record Reader. " + + "Will route message as its own FlowFile to the 'parse.failure' relationship"); + } + + private void handleParseFailure(final ConsumerRecord consumerRecord, final ProcessSession session, final Exception cause, final String message) { + // If we are unable to parse the data, we need to transfer it to 'parse failure' relationship + final Map attributes = getAttributes(consumerRecord); + attributes.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(consumerRecord.offset())); + attributes.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(consumerRecord.partition())); + attributes.put(KafkaProcessorUtils.KAFKA_TOPIC, consumerRecord.topic()); + + FlowFile failureFlowFile = session.create(); + + final byte[] value = consumerRecord.value(); + if (value != null) { + failureFlowFile = session.write(failureFlowFile, out -> out.write(value)); + } + failureFlowFile = session.putAllAttributes(failureFlowFile, attributes); + + final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, consumerRecord.topic()); + session.getProvenanceReporter().receive(failureFlowFile, transitUri); + + session.transfer(failureFlowFile, REL_PARSE_FAILURE); + + if (cause == null) { + logger.error(message); + } else { + logger.error(message, cause); + } + + session.adjustCounter("Parse Failures", 1, false); + } + + private Map getAttributes(final ConsumerRecord consumerRecord) { + final Map attributes = new HashMap<>(); + if (headerNamePattern == null) { + return attributes; + } + + for (final Header header : consumerRecord.headers()) { + final String attributeName = header.key(); + if (headerNamePattern.matcher(attributeName).matches()) { + attributes.put(attributeName, new String(header.value(), headerCharacterSet)); + } + } + + return attributes; + } + + private void writeRecordData(final ProcessSession session, final List> records, final TopicPartition topicPartition) { + // In order to obtain a RecordReader from the RecordReaderFactory, we need to give it a FlowFile. + // We don't want to create a new FlowFile for each record that we receive, so we will just create + // a "temporary flowfile" that will be removed in the finally block below and use that to pass to + // the createRecordReader method. + RecordSetWriter writer = null; + try { + for (final ConsumerRecord consumerRecord : records) { + final Map attributes = getAttributes(consumerRecord); + + final byte[] recordBytes = consumerRecord.value() == null ? new byte[0] : consumerRecord.value(); + try (final InputStream in = new ByteArrayInputStream(recordBytes)) { + final RecordReader reader; + + try { + reader = readerFactory.createRecordReader(attributes, in, logger); + } catch (final Exception e) { + handleParseFailure(consumerRecord, session, e); + continue; + } + + Record record; + while ((record = reader.nextRecord()) != null) { + // Determine the bundle for this record. + final RecordSchema recordSchema = record.getSchema(); + final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes); + + BundleTracker tracker = bundleMap.get(bundleInfo); + if (tracker == null) { + FlowFile flowFile = session.create(); + flowFile = session.putAllAttributes(flowFile, attributes); + + final OutputStream rawOut = session.write(flowFile); + + final RecordSchema writeSchema; + try { + writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema); + } catch (final Exception e) { + logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e); + + try { + rollback(topicPartition); + } catch (final Exception rollbackException) { + logger.warn("Attempted to rollback Kafka message offset but was unable to do so", rollbackException); + } + + yield(); + throw new ProcessException(e); + } + + writer = writerFactory.createWriter(logger, writeSchema, rawOut); + writer.beginRecordSet(); + + tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer); + tracker.updateFlowFile(flowFile); + bundleMap.put(bundleInfo, tracker); + } else { + writer = tracker.recordWriter; + } + + try { + writer.write(record); + } catch (final RuntimeException re) { + handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. " + + "Will route message as its own FlowFile to the 'parse.failure' relationship"); + continue; + } + + tracker.incrementRecordCount(1L); + session.adjustCounter("Records Received", records.size(), false); + } + } + } + } catch (final Exception e) { + logger.error("Failed to properly receive messages from Kafka. Will roll back session and any un-committed offsets from Kafka.", e); + + try { + if (writer != null) { + writer.close(); + } + } catch (final Exception ioe) { + logger.warn("Failed to close Record Writer", ioe); + } + + try { + rollback(topicPartition); + } catch (final Exception rollbackException) { + logger.warn("Attempted to rollback Kafka message offset but was unable to do so", rollbackException); + } + + throw new ProcessException(e); + } + } + + + private void rollback(final TopicPartition topicPartition) { + OffsetAndMetadata offsetAndMetadata = uncommittedOffsetsMap.get(topicPartition); + if (offsetAndMetadata == null) { + offsetAndMetadata = kafkaConsumer.committed(topicPartition); + } + + final long offset = offsetAndMetadata.offset(); + kafkaConsumer.seek(topicPartition, offset); + } + + + + private void populateAttributes(final BundleTracker tracker) { + final Map kafkaAttrs = new HashMap<>(); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset)); + if (tracker.key != null && tracker.totalRecords == 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, tracker.key); + } + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(tracker.partition)); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, tracker.topic); + if (tracker.totalRecords > 1) { + // Add a record.count attribute to remain consistent with other record-oriented processors. If not + // reading/writing records, then use "kafka.count" attribute. + if (tracker.recordWriter == null) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(tracker.totalRecords)); + } else { + kafkaAttrs.put("record.count", String.valueOf(tracker.totalRecords)); + } + } + final FlowFile newFlowFile = getProcessSession().putAllAttributes(tracker.flowFile, kafkaAttrs); + final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos); + final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, tracker.topic); + getProcessSession().getProvenanceReporter().receive(newFlowFile, transitUri, executionDurationMillis); + tracker.updateFlowFile(newFlowFile); + } + + private static class BundleTracker { + + final long initialOffset; + final int partition; + final String topic; + final String key; + final RecordSetWriter recordWriter; + FlowFile flowFile; + long totalRecords = 0; + + private BundleTracker(final ConsumerRecord initialRecord, final TopicPartition topicPartition, final String keyEncoding) { + this(initialRecord, topicPartition, keyEncoding, null); + } + + private BundleTracker(final ConsumerRecord initialRecord, final TopicPartition topicPartition, final String keyEncoding, final RecordSetWriter recordWriter) { + this.initialOffset = initialRecord.offset(); + this.partition = topicPartition.partition(); + this.topic = topicPartition.topic(); + this.recordWriter = recordWriter; + this.key = encodeKafkaKey(initialRecord.key(), keyEncoding); + } + + private void incrementRecordCount(final long count) { + totalRecords += count; + } + + private void updateFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + } + + } + + private static class BundleInformation { + private final TopicPartition topicPartition; + private final RecordSchema schema; + private final Map attributes; + + public BundleInformation(final TopicPartition topicPartition, final RecordSchema schema, final Map attributes) { + this.topicPartition = topicPartition; + this.schema = schema; + this.attributes = attributes; + } + + @Override + public int hashCode() { + return 41 + 13 * topicPartition.hashCode() + ((schema == null) ? 0 : 13 * schema.hashCode()) + ((attributes == null) ? 0 : 13 * attributes.hashCode()); + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof BundleInformation)) { + return false; + } + + final BundleInformation other = (BundleInformation) obj; + return Objects.equals(topicPartition, other.topicPartition) && Objects.equals(schema, other.schema) && Objects.equals(attributes, other.attributes); + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java new file mode 100644 index 0000000000..a7bd96d9df --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java @@ -0,0 +1,372 @@ +/* + * 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.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriterFactory; + +/** + * 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 BlockingQueue pooledLeases; + private final List topics; + private final Pattern topicPattern; + private final Map kafkaProperties; + private final long maxWaitMillis; + private final ComponentLog logger; + private final byte[] demarcatorBytes; + private final String keyEncoding; + private final String securityProtocol; + private final String bootstrapServers; + private final boolean honorTransactions; + private final RecordReaderFactory readerFactory; + private final RecordSetWriterFactory writerFactory; + private final Charset headerCharacterSet; + private final Pattern headerNamePattern; + private final AtomicLong consumerCreatedCountRef = new AtomicLong(); + private final AtomicLong consumerClosedCountRef = new AtomicLong(); + private final AtomicLong leasesObtainedCountRef = new AtomicLong(); + + /** + * Creates a pool of KafkaConsumer objects that will grow up to the maximum + * indicated threads from the given context. Consumers are lazily + * initialized. We may elect to not create up to the maximum number of + * configured consumers if the broker reported lag time for all topics is + * below a certain threshold. + * + * @param maxConcurrentLeases max allowable consumers at once + * @param demarcator bytes to use as demarcator between messages; null or + * empty means no demarcator + * @param kafkaProperties properties to use to initialize kafka consumers + * @param topics the topics to subscribe to + * @param maxWaitMillis maximum time to wait for a given lease to acquire + * data before committing + * @param keyEncoding the encoding to use for the key of a kafka message if + * found + * @param securityProtocol the security protocol used + * @param bootstrapServers the bootstrap servers + * @param logger the logger to report any errors/warnings + */ + public ConsumerPool( + final int maxConcurrentLeases, + final byte[] demarcator, + final Map kafkaProperties, + final List topics, + final long maxWaitMillis, + final String keyEncoding, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger, + final boolean honorTransactions, + final Charset headerCharacterSet, + final Pattern headerNamePattern) { + this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases); + this.maxWaitMillis = maxWaitMillis; + this.logger = logger; + this.demarcatorBytes = demarcator; + this.keyEncoding = keyEncoding; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties); + this.topics = Collections.unmodifiableList(topics); + this.topicPattern = null; + this.readerFactory = null; + this.writerFactory = null; + this.honorTransactions = honorTransactions; + this.headerCharacterSet = headerCharacterSet; + this.headerNamePattern = headerNamePattern; + } + + public ConsumerPool( + final int maxConcurrentLeases, + final byte[] demarcator, + final Map kafkaProperties, + final Pattern topics, + final long maxWaitMillis, + final String keyEncoding, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger, + final boolean honorTransactions, + final Charset headerCharacterSet, + final Pattern headerNamePattern) { + this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases); + this.maxWaitMillis = maxWaitMillis; + this.logger = logger; + this.demarcatorBytes = demarcator; + this.keyEncoding = keyEncoding; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties); + this.topics = null; + this.topicPattern = topics; + this.readerFactory = null; + this.writerFactory = null; + this.honorTransactions = honorTransactions; + this.headerCharacterSet = headerCharacterSet; + this.headerNamePattern = headerNamePattern; + } + + public ConsumerPool( + final int maxConcurrentLeases, + final RecordReaderFactory readerFactory, + final RecordSetWriterFactory writerFactory, + final Map kafkaProperties, + final Pattern topics, + final long maxWaitMillis, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger, + final boolean honorTransactions, + final Charset headerCharacterSet, + final Pattern headerNamePattern) { + this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases); + this.maxWaitMillis = maxWaitMillis; + this.logger = logger; + this.demarcatorBytes = null; + this.keyEncoding = null; + this.readerFactory = readerFactory; + this.writerFactory = writerFactory; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties); + this.topics = null; + this.topicPattern = topics; + this.honorTransactions = honorTransactions; + this.headerCharacterSet = headerCharacterSet; + this.headerNamePattern = headerNamePattern; + } + + public ConsumerPool( + final int maxConcurrentLeases, + final RecordReaderFactory readerFactory, + final RecordSetWriterFactory writerFactory, + final Map kafkaProperties, + final List topics, + final long maxWaitMillis, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger, + final boolean honorTransactions, + final Charset headerCharacterSet, + final Pattern headerNamePattern) { + this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases); + this.maxWaitMillis = maxWaitMillis; + this.logger = logger; + this.demarcatorBytes = null; + this.keyEncoding = null; + this.readerFactory = readerFactory; + this.writerFactory = writerFactory; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties); + this.topics = topics; + this.topicPattern = null; + this.honorTransactions = honorTransactions; + this.headerCharacterSet = headerCharacterSet; + this.headerNamePattern = headerNamePattern; + } + + /** + * Obtains a consumer from the pool if one is available or lazily + * initializes a new one if deemed necessary. + * + * @param session the session for which the consumer lease will be + * associated + * @param processContext the ProcessContext for which the consumer + * lease will be associated + * @return consumer to use or null if not available or necessary + */ + public ConsumerLease obtainConsumer(final ProcessSession session, final ProcessContext processContext) { + SimpleConsumerLease lease = pooledLeases.poll(); + if (lease == null) { + final Consumer consumer = createKafkaConsumer(); + consumerCreatedCountRef.incrementAndGet(); + /** + * For now return a new consumer lease. But we could later elect to + * have this return null if we determine the broker indicates that + * the lag time on all topics being monitored is sufficiently low. + * For now we should encourage conservative use of threads because + * having too many means we'll have at best useless threads sitting + * around doing frequent network calls and at worst having consumers + * sitting idle which could prompt excessive rebalances. + */ + lease = new SimpleConsumerLease(consumer); + /** + * This subscription tightly couples the lease to the given + * consumer. They cannot be separated from then on. + */ + if (topics != null) { + consumer.subscribe(topics, lease); + } else { + consumer.subscribe(topicPattern, lease); + } + } + lease.setProcessSession(session, processContext); + + leasesObtainedCountRef.incrementAndGet(); + return lease; + } + + /** + * Exposed as protected method for easier unit testing + * + * @return consumer + * @throws KafkaException if unable to subscribe to the given topics + */ + protected Consumer createKafkaConsumer() { + final Map properties = new HashMap<>(kafkaProperties); + if (honorTransactions) { + properties.put("isolation.level", "read_committed"); + } else { + properties.put("isolation.level", "read_uncommitted"); + } + final Consumer consumer = new KafkaConsumer<>(properties); + return consumer; + } + + /** + * Closes all consumers in the pool. Can be safely called repeatedly. + */ + @Override + public void close() { + final List leases = new ArrayList<>(); + pooledLeases.drainTo(leases); + leases.stream().forEach((lease) -> { + lease.close(true); + }); + } + + private void closeConsumer(final Consumer consumer) { + consumerClosedCountRef.incrementAndGet(); + 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); + } + } + + PoolStats getPoolStats() { + return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get()); + } + + private class SimpleConsumerLease extends ConsumerLease { + + private final Consumer consumer; + private volatile ProcessSession session; + private volatile ProcessContext processContext; + private volatile boolean closedConsumer; + + private SimpleConsumerLease(final Consumer consumer) { + super(maxWaitMillis, consumer, demarcatorBytes, keyEncoding, securityProtocol, bootstrapServers, + readerFactory, writerFactory, logger, headerCharacterSet, headerNamePattern); + this.consumer = consumer; + } + + void setProcessSession(final ProcessSession session, final ProcessContext context) { + this.session = session; + this.processContext = context; + } + + @Override + public void yield() { + if (processContext != null) { + processContext.yield(); + } + } + + @Override + public ProcessSession getProcessSession() { + return session; + } + + @Override + public void close() { + super.close(); + close(false); + } + + public void close(final boolean forceClose) { + if (closedConsumer) { + return; + } + super.close(); + if (session != null) { + session.rollback(); + setProcessSession(null, null); + } + if (forceClose || isPoisoned() || !pooledLeases.offer(this)) { + closedConsumer = true; + closeConsumer(consumer); + } + } + } + + static final class PoolStats { + + final long consumerCreatedCount; + final long consumerClosedCount; + final long leasesObtainedCount; + + PoolStats( + final long consumerCreatedCount, + final long consumerClosedCount, + final long leasesObtainedCount + ) { + this.consumerCreatedCount = consumerCreatedCount; + this.consumerClosedCount = consumerClosedCount; + this.leasesObtainedCount = leasesObtainedCount; + } + + @Override + public String toString() { + return "Created Consumers [" + consumerCreatedCount + "]\n" + + "Closed Consumers [" + consumerClosedCount + "]\n" + + "Leases Obtained [" + leasesObtainedCount + "]\n"; + } + + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java new file mode 100644 index 0000000000..317b274e8a --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java @@ -0,0 +1,175 @@ +/* + * 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.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; + +public class InFlightMessageTracker { + private final ConcurrentMap messageCountsByFlowFile = new ConcurrentHashMap<>(); + private final ConcurrentMap failures = new ConcurrentHashMap<>(); + private final Object progressMutex = new Object(); + private final ComponentLog logger; + + public InFlightMessageTracker(final ComponentLog logger) { + this.logger = logger; + } + + public void incrementAcknowledgedCount(final FlowFile flowFile) { + final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts()); + counter.incrementAcknowledgedCount(); + + synchronized (progressMutex) { + progressMutex.notify(); + } + } + + public void trackEmpty(final FlowFile flowFile) { + messageCountsByFlowFile.putIfAbsent(flowFile, new Counts()); + } + + public int getAcknowledgedCount(final FlowFile flowFile) { + final Counts counter = messageCountsByFlowFile.get(flowFile); + return (counter == null) ? 0 : counter.getAcknowledgedCount(); + } + + public void incrementSentCount(final FlowFile flowFile) { + final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts()); + counter.incrementSentCount(); + } + + public int getSentCount(final FlowFile flowFile) { + final Counts counter = messageCountsByFlowFile.get(flowFile); + return (counter == null) ? 0 : counter.getSentCount(); + } + + public void fail(final FlowFile flowFile, final Exception exception) { + failures.putIfAbsent(flowFile, exception); + logger.error("Failed to send " + flowFile + " to Kafka", exception); + + synchronized (progressMutex) { + progressMutex.notify(); + } + } + + public Exception getFailure(final FlowFile flowFile) { + return failures.get(flowFile); + } + + public boolean isFailed(final FlowFile flowFile) { + return getFailure(flowFile) != null; + } + + public void reset() { + messageCountsByFlowFile.clear(); + failures.clear(); + } + + public PublishResult failOutstanding(final Exception exception) { + messageCountsByFlowFile.keySet().stream() + .filter(ff -> !isComplete(ff)) + .filter(ff -> !failures.containsKey(ff)) + .forEach(ff -> failures.put(ff, exception)); + + return createPublishResult(); + } + + private boolean isComplete(final FlowFile flowFile) { + final Counts counts = messageCountsByFlowFile.get(flowFile); + if (counts.getAcknowledgedCount() == counts.getSentCount()) { + // all messages received successfully. + return true; + } + + if (failures.containsKey(flowFile)) { + // FlowFile failed so is complete + return true; + } + + return false; + } + + private boolean isComplete() { + return messageCountsByFlowFile.keySet().stream() + .allMatch(flowFile -> isComplete(flowFile)); + } + + void awaitCompletion(final long millis) throws InterruptedException, TimeoutException { + final long startTime = System.nanoTime(); + final long maxTime = startTime + TimeUnit.MILLISECONDS.toNanos(millis); + + while (System.nanoTime() < maxTime) { + synchronized (progressMutex) { + if (isComplete()) { + return; + } + + progressMutex.wait(millis); + } + } + + throw new TimeoutException(); + } + + + PublishResult createPublishResult() { + return new PublishResult() { + @Override + public boolean isFailure() { + return !failures.isEmpty(); + } + + @Override + public int getSuccessfulMessageCount(final FlowFile flowFile) { + return getAcknowledgedCount(flowFile); + } + + @Override + public Exception getReasonForFailure(final FlowFile flowFile) { + return getFailure(flowFile); + } + }; + } + + public static class Counts { + private final AtomicInteger sentCount = new AtomicInteger(0); + private final AtomicInteger acknowledgedCount = new AtomicInteger(0); + + public void incrementSentCount() { + sentCount.incrementAndGet(); + } + + public void incrementAcknowledgedCount() { + acknowledgedCount.incrementAndGet(); + } + + public int getAcknowledgedCount() { + return acknowledgedCount.get(); + } + + public int getSentCount() { + return sentCount.get(); + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java new file mode 100644 index 0000000000..de289959ea --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java @@ -0,0 +1,332 @@ +/* + * 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.commons.lang3.StringUtils; +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()); + + static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string."); + static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded", + "The key is interpreted as arbitrary binary data and is encoded using hexadecimal characters with uppercase letters"); + + static final Pattern HEX_KEY_PATTERN = Pattern.compile("(?:[0123456789abcdefABCDEF]{2})+"); + + static final String KAFKA_KEY = "kafka.key"; + 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.HOSTNAME_PORT_LIST_VALIDATOR) + .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 USER_PRINCIPAL = new PropertyDescriptor.Builder() + .name("sasl.kerberos.principal") + .displayName("Kerberos Principal") + .description("The Kerberos principal that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file " + + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.") + .required(false) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + static final PropertyDescriptor USER_KEYTAB = new PropertyDescriptor.Builder() + .name("sasl.kerberos.keytab") + .displayName("Kerberos Keytab") + .description("The Kerberos keytab that will be used to connect to brokers. If not set, it is expected to set a JAAS configuration file " + + "in the JVM properties defined in the bootstrap.conf file. This principal will be set into 'sasl.jaas.config' Kafka's property.") + .required(false) + .addValidator(StandardValidators.FILE_EXISTS_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, + USER_PRINCIPAL, + USER_KEYTAB, + 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()); + } + + String userKeytab = validationContext.getProperty(USER_KEYTAB).getValue(); + String userPrincipal = validationContext.getProperty(USER_PRINCIPAL).getValue(); + if((StringUtils.isBlank(userKeytab) && !StringUtils.isBlank(userPrincipal)) + || (!StringUtils.isBlank(userKeytab) && StringUtils.isBlank(userPrincipal))) { + results.add(new ValidationResult.Builder().subject(KERBEROS_PRINCIPLE.getDisplayName()).valid(false) + .explanation("Both <" + USER_KEYTAB.getDisplayName() + "> and <" + USER_PRINCIPAL.getDisplayName() + "> " + + "must be set.") + .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 propertyName = propertyDescriptor.getName(); + String propertyValue = propertyDescriptor.isExpressionLanguageSupported() + ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue() + : context.getProperty(propertyDescriptor).getValue(); + + if (propertyValue != null && !propertyName.equals(USER_PRINCIPAL.getName()) && !propertyName.equals(USER_KEYTAB.getName())) { + // If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds + // or the standard NiFi time period such as "5 secs" + if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation + propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS)); + } + + if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) { + mapToPopulate.put(propertyName, propertyValue); + } + } + } + + String securityProtocol = context.getProperty(SECURITY_PROTOCOL).getValue(); + if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)) { + setJaasConfig(mapToPopulate, context); + } + } + + /** + * Method used to configure the 'sasl.jaas.config' property based on KAFKA-4259
+ * https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients
+ *
+ * It expects something with the following format:
+ *
+ * <LoginModuleClass> <ControlFlag> *(<OptionName>=<OptionValue>);
+ * ControlFlag = required / requisite / sufficient / optional + * + * @param mapToPopulate Map of configuration properties + * @param context Context + */ + private static void setJaasConfig(Map mapToPopulate, ProcessContext context) { + String keytab = context.getProperty(USER_KEYTAB).getValue(); + String principal = context.getProperty(USER_PRINCIPAL).getValue(); + String serviceName = context.getProperty(KERBEROS_PRINCIPLE).getValue(); + if(StringUtils.isNotBlank(keytab) && StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(serviceName)) { + mapToPopulate.put(SaslConfigs.SASL_JAAS_CONFIG, "com.sun.security.auth.module.Krb5LoginModule required " + + "useTicketCache=false " + + "renewTicket=true " + + "serviceName=\"" + serviceName + "\" " + + "useKeyTab=true " + + "keyTab=\"" + keytab + "\" " + + "principal=\"" + principal + "\";"); + } + } + + 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-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/Partitioners.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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-1-0-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-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java new file mode 100644 index 0000000000..c125d622ba --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_1_0.java @@ -0,0 +1,443 @@ +/* + * 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.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +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.Set; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +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.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +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.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +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.io.InputStreamCallback; +import org.apache.nifi.processor.util.FlowFileFilters; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordSet; + +@Tags({"Apache", "Kafka", "Record", "csv", "json", "avro", "logs", "Put", "Send", "Message", "PubSub", "1.0"}) +@CapabilityDescription("Sends the contents of a FlowFile as individual records to Apache Kafka using the Kafka 1.0 Producer API. " + + "The contents of the FlowFile are expected to be record-oriented data that can be read by the configured Record Reader. " + + "The complementary NiFi processor for fetching messages is ConsumeKafkaRecord_1_0.") +@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. ") +@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to " + + "FlowFiles that are routed to success.") +@SeeAlso({PublishKafka_1_0.class, ConsumeKafka_1_0.class, ConsumeKafkaRecord_1_0.class}) +public class PublishKafkaRecord_1_0 extends AbstractProcessor { + 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 AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string."); + static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded", + "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters."); + + 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 RECORD_READER = new PropertyDescriptor.Builder() + .name("record-reader") + .displayName("Record Reader") + .description("The Record Reader to use for incoming FlowFiles") + .identifiesControllerService(RecordReaderFactory.class) + .expressionLanguageSupported(false) + .required(true) + .build(); + + static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() + .name("record-writer") + .displayName("Record Writer") + .description("The Record Writer to use in order to serialize the data before sending to Kafka") + .identifiesControllerService(RecordSetWriterFactory.class) + .expressionLanguageSupported(false) + .required(true) + .build(); + + static final PropertyDescriptor MESSAGE_KEY_FIELD = new PropertyDescriptor.Builder() + .name("message-key-field") + .displayName("Message Key Field") + .description("The name of a field in the Input Records that should be used as the Key for the Kafka message.") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .required(false) + .build(); + + static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder() + .name("acks") + .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 METADATA_WAIT_TIME = new PropertyDescriptor.Builder() + .name("max.block.ms") + .displayName("Max Metadata Wait Time") + .description("The amount of time publisher will wait to obtain metadata or wait for the buffer to flush 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("5 sec") + .build(); + + static final PropertyDescriptor ACK_WAIT_TIME = new PropertyDescriptor.Builder() + .name("ack.wait.time") + .displayName("Acknowledgment Wait Time") + .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. " + + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .required(true) + .defaultValue("5 secs") + .build(); + + static final PropertyDescriptor MAX_REQUEST_SIZE = new PropertyDescriptor.Builder() + .name("max.request.size") + .displayName("Max Request Size") + .description("The maximum size of a request in bytes. Corresponds to Kafka's 'max.request.size' property and defaults to 1 MB (1048576).") + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .defaultValue("1 MB") + .build(); + + static final PropertyDescriptor PARTITION_CLASS = new PropertyDescriptor.Builder() + .name("partitioner.class") + .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("compression.type") + .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 PropertyDescriptor ATTRIBUTE_NAME_REGEX = new PropertyDescriptor.Builder() + .name("attribute-name-regex") + .displayName("Attributes to Send as Headers (Regex)") + .description("A Regular Expression that is matched against all FlowFile attribute names. " + + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. " + + "If not specified, no FlowFile attributes will be added as headers.") + .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) + .expressionLanguageSupported(false) + .required(false) + .build(); + static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder() + .name("use-transactions") + .displayName("Use Transactions") + .description("Specifies whether or not NiFi should provide Transactional guarantees when communicating with Kafka. If there is a problem sending data to Kafka, " + + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. " + + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true " + + "requires that the property be set to \"Guarantee Replicated Delivery.\"") + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); + static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder() + .name("message-header-encoding") + .displayName("Message Header Encoding") + .description("For any attribute that is added as a message header, as configured via the property, " + + "this property indicates the Character Encoding to use for serializing the headers.") + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .required(false) + .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(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + private volatile PublisherPool publisherPool = null; + + static { + final List properties = new ArrayList<>(); + properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS); + properties.add(TOPIC); + properties.add(RECORD_READER); + properties.add(RECORD_WRITER); + properties.add(USE_TRANSACTIONS); + properties.add(DELIVERY_GUARANTEE); + properties.add(ATTRIBUTE_NAME_REGEX); + properties.add(MESSAGE_HEADER_ENCODING); + properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL); + properties.add(KafkaProcessorUtils.KERBEROS_PRINCIPLE); + properties.add(KafkaProcessorUtils.USER_PRINCIPAL); + properties.add(KafkaProcessorUtils.USER_KEYTAB); + properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE); + properties.add(MESSAGE_KEY_FIELD); + properties.add(MAX_REQUEST_SIZE); + properties.add(ACK_WAIT_TIME); + properties.add(METADATA_WAIT_TIME); + properties.add(PARTITION_CLASS); + properties.add(COMPRESSION_CODEC); + + PROPERTIES = Collections.unmodifiableList(properties); + + 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 PROPERTIES; + } + + @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) { + final List results = new ArrayList<>(); + results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext)); + + final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean(); + if (useTransactions) { + final String deliveryGuarantee = validationContext.getProperty(DELIVERY_GUARANTEE).getValue(); + if (!DELIVERY_REPLICATED.getValue().equals(deliveryGuarantee)) { + results.add(new ValidationResult.Builder() + .subject("Delivery Guarantee") + .valid(false) + .explanation("In order to use Transactions, the Delivery Guarantee must be \"Guarantee Replicated Delivery.\" " + + "Either change the property or the property.") + .build()); + } + } + + return results; + } + + private synchronized PublisherPool getPublisherPool(final ProcessContext context) { + PublisherPool pool = publisherPool; + if (pool != null) { + return pool; + } + + return publisherPool = createPublisherPool(context); + } + + protected PublisherPool createPublisherPool(final ProcessContext context) { + final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue(); + final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue(); + + final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue(); + final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex); + final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean(); + + final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue(); + final Charset charset = Charset.forName(charsetName); + + 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()); + kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize)); + + return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset); + } + + @OnStopped + public void closePool() { + if (publisherPool != null) { + publisherPool.close(); + } + + publisherPool = null; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + final List flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(1, DataUnit.MB, 500)); + if (flowFiles.isEmpty()) { + return; + } + + final PublisherPool pool = getPublisherPool(context); + if (pool == null) { + context.yield(); + return; + } + + final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(); + final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class); + final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); + final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean(); + + final long startTime = System.nanoTime(); + try (final PublisherLease lease = pool.obtainPublisher()) { + if (useTransactions) { + lease.beginTransaction(); + } + + // Send each FlowFile to Kafka asynchronously. + for (final FlowFile flowFile : flowFiles) { + if (!isScheduled()) { + // If stopped, re-queue FlowFile instead of sending it + if (useTransactions) { + session.rollback(); + lease.rollback(); + return; + } + + session.transfer(flowFile); + continue; + } + + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + final String messageKeyField = context.getProperty(MESSAGE_KEY_FIELD).evaluateAttributeExpressions(flowFile).getValue(); + + try { + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream rawIn) throws IOException { + try (final InputStream in = new BufferedInputStream(rawIn)) { + final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger()); + final RecordSet recordSet = reader.createRecordSet(); + + final RecordSchema schema = writerFactory.getSchema(flowFile.getAttributes(), recordSet.getSchema()); + lease.publish(flowFile, recordSet, writerFactory, schema, messageKeyField, topic); + } catch (final SchemaNotFoundException | MalformedRecordException e) { + throw new ProcessException(e); + } + } + }); + } catch (final Exception e) { + // The FlowFile will be obtained and the error logged below, when calling publishResult.getFailedFlowFiles() + lease.fail(flowFile, e); + continue; + } + } + + // Complete the send + final PublishResult publishResult = lease.complete(); + + if (publishResult.isFailure()) { + getLogger().info("Failed to send FlowFile to kafka; transferring to failure"); + session.transfer(flowFiles, REL_FAILURE); + return; + } + + // Transfer any successful FlowFiles. + final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime); + for (FlowFile success : flowFiles) { + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue(); + + final int msgCount = publishResult.getSuccessfulMessageCount(success); + success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount)); + session.adjustCounter("Messages Sent", msgCount, true); + + final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic); + session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis); + session.transfer(success, REL_SUCCESS); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java new file mode 100644 index 0000000000..6f17bd5fa5 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_1_0.java @@ -0,0 +1,458 @@ +/* + * 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.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +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.Set; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +import javax.xml.bind.DatatypeConverter; + +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.WritesAttribute; +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.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +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.io.InputStreamCallback; +import org.apache.nifi.processor.util.FlowFileFilters; +import org.apache.nifi.processor.util.StandardValidators; + +@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "1.0"}) +@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 1.0 Producer API." + + "The messages to send may be individual FlowFiles or may be delimited, using a " + + "user-specified delimiter, such as a new-line. " + + "The complementary NiFi processor for fetching messages is ConsumeKafka_1_0.") +@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. ") +@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to " + + "FlowFiles that are routed to success. If the Property is not set, this will always be 1, but if the Property is set, it may " + + "be greater than 1.") +public class PublishKafka_1_0 extends AbstractProcessor { + 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 AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string."); + static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded", + "The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters."); + + 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 METADATA_WAIT_TIME = new PropertyDescriptor.Builder() + .name(ProducerConfig.MAX_BLOCK_MS_CONFIG) + .displayName("Max Metadata Wait Time") + .description("The amount of time publisher will wait to obtain metadata or wait for the buffer to flush 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("5 sec") + .build(); + + static final PropertyDescriptor ACK_WAIT_TIME = new PropertyDescriptor.Builder() + .name("ack.wait.time") + .displayName("Acknowledgment Wait Time") + .description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. " + + "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .required(true) + .defaultValue("5 secs") + .build(); + + static final PropertyDescriptor MAX_REQUEST_SIZE = new PropertyDescriptor.Builder() + .name("max.request.size") + .displayName("Max Request Size") + .description("The maximum size of a request in bytes. Corresponds to Kafka's 'max.request.size' property and defaults to 1 MB (1048576).") + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .defaultValue("1 MB") + .build(); + + static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() + .name("kafka-key") + .displayName("Kafka Key") + .description("The Key to use for the Message. " + + "If not specified, the flow file attribute 'kafka.key' is used as the message key, if it is present " + + "and we're not demarcating.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + static final PropertyDescriptor KEY_ATTRIBUTE_ENCODING = new PropertyDescriptor.Builder() + .name("key-attribute-encoding") + .displayName("Key Attribute Encoding") + .description("FlowFiles that are emitted have an attribute named '" + KafkaProcessorUtils.KAFKA_KEY + "'. This property dictates how the value of the attribute should be encoded.") + .required(true) + .defaultValue(UTF8_ENCODING.getValue()) + .allowableValues(UTF8_ENCODING, HEX_ENCODING) + .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 PropertyDescriptor ATTRIBUTE_NAME_REGEX = new PropertyDescriptor.Builder() + .name("attribute-name-regex") + .displayName("Attributes to Send as Headers (Regex)") + .description("A Regular Expression that is matched against all FlowFile attribute names. " + + "Any attribute whose name matches the regex will be added to the Kafka messages as a Header. " + + "If not specified, no FlowFile attributes will be added as headers.") + .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) + .expressionLanguageSupported(false) + .required(false) + .build(); + static final PropertyDescriptor USE_TRANSACTIONS = new PropertyDescriptor.Builder() + .name("use-transactions") + .displayName("Use Transactions") + .description("Specifies whether or not NiFi should provide Transactional guarantees when communicating with Kafka. If there is a problem sending data to Kafka, " + + "and this property is set to false, then the messages that have already been sent to Kafka will continue on and be delivered to consumers. " + + "If this is set to true, then the Kafka transaction will be rolled back so that those messages are not available to consumers. Setting this to true " + + "requires that the property be set to \"Guarantee Replicated Delivery.\"") + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); + static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder() + .name("message-header-encoding") + .displayName("Message Header Encoding") + .description("For any attribute that is added as a message header, as configured via the property, " + + "this property indicates the Character Encoding to use for serializing the headers.") + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .required(false) + .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(); + + private static final List PROPERTIES; + private static final Set RELATIONSHIPS; + + private volatile PublisherPool publisherPool = null; + + static { + final List properties = new ArrayList<>(); + properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors()); + properties.add(TOPIC); + properties.add(DELIVERY_GUARANTEE); + properties.add(USE_TRANSACTIONS); + properties.add(ATTRIBUTE_NAME_REGEX); + properties.add(MESSAGE_HEADER_ENCODING); + properties.add(KEY); + properties.add(KEY_ATTRIBUTE_ENCODING); + properties.add(MESSAGE_DEMARCATOR); + properties.add(MAX_REQUEST_SIZE); + properties.add(ACK_WAIT_TIME); + properties.add(METADATA_WAIT_TIME); + properties.add(PARTITION_CLASS); + properties.add(COMPRESSION_CODEC); + + PROPERTIES = Collections.unmodifiableList(properties); + + 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 PROPERTIES; + } + + @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) { + final List results = new ArrayList<>(); + results.addAll(KafkaProcessorUtils.validateCommonProperties(validationContext)); + + final boolean useTransactions = validationContext.getProperty(USE_TRANSACTIONS).asBoolean(); + if (useTransactions) { + final String deliveryGuarantee = validationContext.getProperty(DELIVERY_GUARANTEE).getValue(); + if (!DELIVERY_REPLICATED.getValue().equals(deliveryGuarantee)) { + results.add(new ValidationResult.Builder() + .subject("Delivery Guarantee") + .valid(false) + .explanation("In order to use Transactions, the Delivery Guarantee must be \"Guarantee Replicated Delivery.\" " + + "Either change the property or the property.") + .build()); + } + } + + return results; + } + + private synchronized PublisherPool getPublisherPool(final ProcessContext context) { + PublisherPool pool = publisherPool; + if (pool != null) { + return pool; + } + + return publisherPool = createPublisherPool(context); + } + + protected PublisherPool createPublisherPool(final ProcessContext context) { + final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue(); + final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue(); + + final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue(); + final Pattern attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex); + final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean(); + + final String charsetName = context.getProperty(MESSAGE_HEADER_ENCODING).evaluateAttributeExpressions().getValue(); + final Charset charset = Charset.forName(charsetName); + + 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()); + kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize)); + + return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis, useTransactions, attributeNamePattern, charset); + } + + @OnStopped + public void closePool() { + if (publisherPool != null) { + publisherPool.close(); + } + + publisherPool = null; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet(); + + final List flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500)); + if (flowFiles.isEmpty()) { + return; + } + + final PublisherPool pool = getPublisherPool(context); + if (pool == null) { + context.yield(); + return; + } + + final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(); + final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); + final boolean useTransactions = context.getProperty(USE_TRANSACTIONS).asBoolean(); + + final long startTime = System.nanoTime(); + try (final PublisherLease lease = pool.obtainPublisher()) { + if (useTransactions) { + lease.beginTransaction(); + } + + // Send each FlowFile to Kafka asynchronously. + for (final FlowFile flowFile : flowFiles) { + if (!isScheduled()) { + // If stopped, re-queue FlowFile instead of sending it + if (useTransactions) { + session.rollback(); + lease.rollback(); + return; + } + + session.transfer(flowFile); + continue; + } + + final byte[] messageKey = getMessageKey(flowFile, context); + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); + final byte[] demarcatorBytes; + if (useDemarcator) { + demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8); + } else { + demarcatorBytes = null; + } + + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream rawIn) throws IOException { + try (final InputStream in = new BufferedInputStream(rawIn)) { + lease.publish(flowFile, in, messageKey, demarcatorBytes, topic); + } + } + }); + } + + // Complete the send + final PublishResult publishResult = lease.complete(); + + if (publishResult.isFailure()) { + getLogger().info("Failed to send FlowFile to kafka; transferring to failure"); + session.transfer(flowFiles, REL_FAILURE); + return; + } + + // Transfer any successful FlowFiles. + final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime); + for (FlowFile success : flowFiles) { + final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue(); + + final int msgCount = publishResult.getSuccessfulMessageCount(success); + success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount)); + session.adjustCounter("Messages Sent", msgCount, true); + + final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic); + session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis); + session.transfer(success, REL_SUCCESS); + } + } + } + + + private byte[] getMessageKey(final FlowFile flowFile, final ProcessContext context) { + if (context.getProperty(MESSAGE_DEMARCATOR).isSet()) { + return null; + } + + final String uninterpretedKey; + if (context.getProperty(KEY).isSet()) { + uninterpretedKey = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); + } else { + uninterpretedKey = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_KEY); + } + + if (uninterpretedKey == null) { + return null; + } + + final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue(); + if (UTF8_ENCODING.getValue().equals(keyEncoding)) { + return uninterpretedKey.getBytes(StandardCharsets.UTF_8); + } + + return DatatypeConverter.parseHexBinary(uninterpretedKey); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java new file mode 100644 index 0000000000..1f7c3ab0c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java @@ -0,0 +1,46 @@ +/* + * 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.nifi.flowfile.FlowFile; + +public interface PublishResult { + + boolean isFailure(); + + int getSuccessfulMessageCount(FlowFile flowFile); + + Exception getReasonForFailure(FlowFile flowFile); + + public static PublishResult EMPTY = new PublishResult() { + @Override + public boolean isFailure() { + return false; + } + + @Override + public int getSuccessfulMessageCount(FlowFile flowFile) { + return 0; + } + + @Override + public Exception getReasonForFailure(FlowFile flowFile) { + return null; + } + }; +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java new file mode 100644 index 0000000000..abcd15f249 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.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.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.header.Headers; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordSet; +import org.apache.nifi.stream.io.exception.TokenTooLargeException; +import org.apache.nifi.stream.io.util.StreamDemarcator; + +public class PublisherLease implements Closeable { + private final ComponentLog logger; + private final Producer producer; + private final int maxMessageSize; + private final long maxAckWaitMillis; + private final boolean useTransactions; + private final Pattern attributeNameRegex; + private final Charset headerCharacterSet; + private volatile boolean poisoned = false; + private final AtomicLong messagesSent = new AtomicLong(0L); + + private volatile boolean transactionsInitialized = false; + private volatile boolean activeTransaction = false; + + private InFlightMessageTracker tracker; + + public PublisherLease(final Producer producer, final int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger, + final boolean useTransactions, final Pattern attributeNameRegex, final Charset headerCharacterSet) { + this.producer = producer; + this.maxMessageSize = maxMessageSize; + this.logger = logger; + this.maxAckWaitMillis = maxAckWaitMillis; + this.useTransactions = useTransactions; + this.attributeNameRegex = attributeNameRegex; + this.headerCharacterSet = headerCharacterSet; + } + + protected void poison() { + this.poisoned = true; + } + + public boolean isPoisoned() { + return poisoned; + } + + void beginTransaction() { + if (!useTransactions) { + return; + } + + if (!transactionsInitialized) { + producer.initTransactions(); + transactionsInitialized = true; + } + + producer.beginTransaction(); + activeTransaction = true; + } + + void rollback() { + if (!useTransactions || !activeTransaction) { + return; + } + + producer.abortTransaction(); + activeTransaction = false; + } + + void fail(final FlowFile flowFile, final Exception cause) { + getTracker().fail(flowFile, cause); + rollback(); + } + + void publish(final FlowFile flowFile, final InputStream flowFileContent, final byte[] messageKey, final byte[] demarcatorBytes, final String topic) throws IOException { + if (tracker == null) { + tracker = new InFlightMessageTracker(logger); + } + + try (final StreamDemarcator demarcator = new StreamDemarcator(flowFileContent, demarcatorBytes, maxMessageSize)) { + byte[] messageContent; + try { + while ((messageContent = demarcator.nextToken()) != null) { + // We do not want to use any key if we have a demarcator because that would result in + // the key being the same for multiple messages + final byte[] keyToUse = demarcatorBytes == null ? messageKey : null; + publish(flowFile, keyToUse, messageContent, topic, tracker); + + if (tracker.isFailed(flowFile)) { + // If we have a failure, don't try to send anything else. + return; + } + } + } catch (final TokenTooLargeException ttle) { + tracker.fail(flowFile, ttle); + } + } catch (final Exception e) { + tracker.fail(flowFile, e); + poison(); + throw e; + } + } + + void publish(final FlowFile flowFile, final RecordSet recordSet, final RecordSetWriterFactory writerFactory, final RecordSchema schema, + final String messageKeyField, final String topic) throws IOException { + if (tracker == null) { + tracker = new InFlightMessageTracker(logger); + } + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); + + Record record; + int recordCount = 0; + + try (final RecordSetWriter writer = writerFactory.createWriter(logger, schema, baos)) { + while ((record = recordSet.next()) != null) { + recordCount++; + baos.reset(); + + writer.write(record); + writer.flush(); + + final byte[] messageContent = baos.toByteArray(); + final String key = messageKeyField == null ? null : record.getAsString(messageKeyField); + final byte[] messageKey = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8); + + publish(flowFile, messageKey, messageContent, topic, tracker); + + if (tracker.isFailed(flowFile)) { + // If we have a failure, don't try to send anything else. + return; + } + } + + if (recordCount == 0) { + tracker.trackEmpty(flowFile); + } + } catch (final TokenTooLargeException ttle) { + tracker.fail(flowFile, ttle); + } catch (final SchemaNotFoundException snfe) { + throw new IOException(snfe); + } catch (final Exception e) { + tracker.fail(flowFile, e); + poison(); + throw e; + } + } + + private void addHeaders(final FlowFile flowFile, final ProducerRecord record) { + if (attributeNameRegex == null) { + return; + } + + final Headers headers = record.headers(); + for (final Map.Entry entry : flowFile.getAttributes().entrySet()) { + if (attributeNameRegex.matcher(entry.getKey()).matches()) { + headers.add(entry.getKey(), entry.getValue().getBytes(headerCharacterSet)); + } + } + } + + protected void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker) { + final ProducerRecord record = new ProducerRecord<>(topic, null, messageKey, messageContent); + addHeaders(flowFile, record); + + producer.send(record, new Callback() { + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception == null) { + tracker.incrementAcknowledgedCount(flowFile); + } else { + tracker.fail(flowFile, exception); + poison(); + } + } + }); + + messagesSent.incrementAndGet(); + tracker.incrementSentCount(flowFile); + } + + + public PublishResult complete() { + if (tracker == null) { + if (messagesSent.get() == 0L) { + return PublishResult.EMPTY; + } + + rollback(); + throw new IllegalStateException("Cannot complete publishing to Kafka because Publisher Lease was already closed"); + } + + producer.flush(); + + if (activeTransaction) { + producer.commitTransaction(); + activeTransaction = false; + } + + try { + tracker.awaitCompletion(maxAckWaitMillis); + return tracker.createPublishResult(); + } catch (final InterruptedException e) { + logger.warn("Interrupted while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka"); + Thread.currentThread().interrupt(); + return tracker.failOutstanding(e); + } catch (final TimeoutException e) { + logger.warn("Timed out while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka"); + return tracker.failOutstanding(e); + } finally { + tracker = null; + } + } + + @Override + public void close() { + producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS); + tracker = null; + } + + public InFlightMessageTracker getTracker() { + if (tracker == null) { + tracker = new InFlightMessageTracker(logger); + } + + return tracker; + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java new file mode 100644 index 0000000000..d5caa8d5d4 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java @@ -0,0 +1,115 @@ +/* + * 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.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.regex.Pattern; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.nifi.logging.ComponentLog; + +public class PublisherPool implements Closeable { + private final ComponentLog logger; + private final BlockingQueue publisherQueue; + private final Map kafkaProperties; + private final int maxMessageSize; + private final long maxAckWaitMillis; + private final boolean useTransactions; + private final Pattern attributeNameRegex; + private final Charset headerCharacterSet; + + private volatile boolean closed = false; + + PublisherPool(final Map kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis, + final boolean useTransactions, final Pattern attributeNameRegex, final Charset headerCharacterSet) { + this.logger = logger; + this.publisherQueue = new LinkedBlockingQueue<>(); + this.kafkaProperties = kafkaProperties; + this.maxMessageSize = maxMessageSize; + this.maxAckWaitMillis = maxAckWaitMillis; + this.useTransactions = useTransactions; + this.attributeNameRegex = attributeNameRegex; + this.headerCharacterSet = headerCharacterSet; + } + + public PublisherLease obtainPublisher() { + if (isClosed()) { + throw new IllegalStateException("Connection Pool is closed"); + } + + PublisherLease lease = publisherQueue.poll(); + if (lease != null) { + return lease; + } + + lease = createLease(); + return lease; + } + + private PublisherLease createLease() { + final Map properties = new HashMap<>(kafkaProperties); + if (useTransactions) { + properties.put("transactional.id", UUID.randomUUID().toString()); + } + + final Producer producer = new KafkaProducer<>(properties); + + final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger, useTransactions, attributeNameRegex, headerCharacterSet) { + @Override + public void close() { + if (isPoisoned() || isClosed()) { + super.close(); + } else { + publisherQueue.offer(this); + } + } + }; + + return lease; + } + + public synchronized boolean isClosed() { + return closed; + } + + @Override + public synchronized void close() { + closed = true; + + PublisherLease lease; + while ((lease = publisherQueue.poll()) != null) { + lease.close(); + } + } + + /** + * Returns the number of leases that are currently available + * + * @return the number of leases currently available + */ + protected int available() { + return publisherQueue.size(); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..ea9d84dd54 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,18 @@ +# 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_1_0 +org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_1_0 +org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_1_0 +org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0/additionalDetails.html b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0/additionalDetails.html new file mode 100644 index 0000000000..1fd64496c9 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0/additionalDetails.html @@ -0,0 +1,143 @@ + + + + + + ConsumeKafka + + + + +

Description

+

+ This Processor polls Apache Kafka + for data using KafkaConsumer API available with Kafka 1.0. When a message is received + from Kafka, the message will be deserialized using the configured Record Reader, and then + written to a FlowFile by serializing the message with the configured Record Writer. +

+ + +

Security Configuration:

+

+ The Security Protocol property allows the user to specify the protocol for communicating + with the Kafka broker. The following sections describe each of the protocols in further detail. +

+

PLAINTEXT

+

+ This option provides an unsecured connection to the broker, with no client authentication and no encryption. + In order to use this option the broker must be configured with a listener of the form: +

+    PLAINTEXT://host.name:port
+            
+

+

SSL

+

+ This option provides an encrypted connection to the broker, with optional client authentication. In order + to use this option the broker must be configured with a listener of the form: +

+    SSL://host.name:port
+            
+ In addition, the processor must have an SSL Context Service selected. +

+

+ If the broker specifies ssl.client.auth=none, or does not specify ssl.client.auth, then the client will + not be required to present a certificate. In this case, the SSL Context Service selected may specify only + a truststore containing the public key of the certificate authority used to sign the broker's key. +

+

+ If the broker specifies ssl.client.auth=required then the client will be required to present a certificate. + In this case, the SSL Context Service must also specify a keystore containing a client key, in addition to + a truststore as described above. +

+

SASL_PLAINTEXT

+

+ This option uses SASL with a PLAINTEXT transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_PLAINTEXT://host.name:port
+            
+ In addition, the Kerberos Service Name must be specified in the processor. +

+

SASL_PLAINTEXT - GSSAPI

+

+ If the SASL mechanism is GSSAPI, then the client must provide a JAAS configuration to authenticate. The + JAAS configuration can be provided by specifying the java.security.auth.login.config system property in + NiFi's bootstrap.conf, such as: +

+    java.arg.16=-Djava.security.auth.login.config=/path/to/kafka_client_jaas.conf
+            
+

+

+ An example of the JAAS config file would be the following: +

+    KafkaClient {
+        com.sun.security.auth.module.Krb5LoginModule required
+        useKeyTab=true
+        storeKey=true
+        keyTab="/path/to/nifi.keytab"
+        serviceName="kafka"
+        principal="nifi@YOURREALM.COM";
+    };
+            
+ NOTE: The serviceName in the JAAS file must match the Kerberos Service Name in the processor. +

+

+ Alternatively, the JAAS + configuration when using GSSAPI can be provided by specifying the Kerberos Principal and Kerberos Keytab + directly in the processor properties. This will dynamically create a JAAS configuration like above, and + will take precedence over the java.security.auth.login.config system property. +

+

SASL_PLAINTEXT - PLAIN

+

+ If the SASL mechanism is PLAIN, then client must provide a JAAS configuration to authenticate, but + the JAAS configuration must use Kafka's PlainLoginModule. An example of the JAAS config file would + be the following: +

+    KafkaClient {
+      org.apache.kafka.common.security.plain.PlainLoginModule required
+      username="nifi"
+      password="nifi-password";
+    };
+            
+

+

+ NOTE: It is not recommended to use a SASL mechanism of PLAIN with SASL_PLAINTEXT, as it would transmit + the username and password unencrypted. +

+

+ NOTE: Using the PlainLoginModule will cause it be registered in the JVM's static list of Providers, making + it visible to components in other NARs that may access the providers. There is currently a known issue + where Kafka processors using the PlainLoginModule will cause HDFS processors with Keberos to no longer work. +

+

SASL_SSL

+

+ This option uses SASL with an SSL/TLS transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_SSL://host.name:port
+            
+

+

+ See the SASL_PLAINTEXT section for a description of how to provide the proper JAAS configuration + depending on the SASL mechanism (GSSAPI or PLAIN). +

+

+ See the SSL section for a description of how to configure the SSL Context Service based on the + ssl.client.auth property. +

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

Description

+

+ This Processor polls Apache Kafka + for data using KafkaConsumer API available with Kafka 1.0. 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. +

+ + +

Security Configuration

+

+ The Security Protocol property allows the user to specify the protocol for communicating + with the Kafka broker. The following sections describe each of the protocols in further detail. +

+

PLAINTEXT

+

+ This option provides an unsecured connection to the broker, with no client authentication and no encryption. + In order to use this option the broker must be configured with a listener of the form: +

+    PLAINTEXT://host.name:port
+            
+

+

SSL

+

+ This option provides an encrypted connection to the broker, with optional client authentication. In order + to use this option the broker must be configured with a listener of the form: +

+    SSL://host.name:port
+            
+ In addition, the processor must have an SSL Context Service selected. +

+

+ If the broker specifies ssl.client.auth=none, or does not specify ssl.client.auth, then the client will + not be required to present a certificate. In this case, the SSL Context Service selected may specify only + a truststore containing the public key of the certificate authority used to sign the broker's key. +

+

+ If the broker specifies ssl.client.auth=required then the client will be required to present a certificate. + In this case, the SSL Context Service must also specify a keystore containing a client key, in addition to + a truststore as described above. +

+

SASL_PLAINTEXT

+

+ This option uses SASL with a PLAINTEXT transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_PLAINTEXT://host.name:port
+            
+ In addition, the Kerberos Service Name must be specified in the processor. +

+

SASL_PLAINTEXT - GSSAPI

+

+ If the SASL mechanism is GSSAPI, then the client must provide a JAAS configuration to authenticate. The + JAAS configuration can be provided by specifying the java.security.auth.login.config system property in + NiFi's bootstrap.conf, such as: +

+    java.arg.16=-Djava.security.auth.login.config=/path/to/kafka_client_jaas.conf
+            
+

+

+ An example of the JAAS config file would be the following: +

+    KafkaClient {
+        com.sun.security.auth.module.Krb5LoginModule required
+        useKeyTab=true
+        storeKey=true
+        keyTab="/path/to/nifi.keytab"
+        serviceName="kafka"
+        principal="nifi@YOURREALM.COM";
+    };
+            
+ NOTE: The serviceName in the JAAS file must match the Kerberos Service Name in the processor. +

+

+ Alternatively, the JAAS + configuration when using GSSAPI can be provided by specifying the Kerberos Principal and Kerberos Keytab + directly in the processor properties. This will dynamically create a JAAS configuration like above, and + will take precedence over the java.security.auth.login.config system property. +

+

SASL_PLAINTEXT - PLAIN

+

+ If the SASL mechanism is PLAIN, then client must provide a JAAS configuration to authenticate, but + the JAAS configuration must use Kafka's PlainLoginModule. An example of the JAAS config file would + be the following: +

+    KafkaClient {
+      org.apache.kafka.common.security.plain.PlainLoginModule required
+      username="nifi"
+      password="nifi-password";
+    };
+            
+

+

+ NOTE: It is not recommended to use a SASL mechanism of PLAIN with SASL_PLAINTEXT, as it would transmit + the username and password unencrypted. +

+

+ NOTE: Using the PlainLoginModule will cause it be registered in the JVM's static list of Providers, making + it visible to components in other NARs that may access the providers. There is currently a known issue + where Kafka processors using the PlainLoginModule will cause HDFS processors with Keberos to no longer work. +

+

SASL_SSL

+

+ This option uses SASL with an SSL/TLS transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_SSL://host.name:port
+            
+

+

+ See the SASL_PLAINTEXT section for a description of how to provide the proper JAAS configuration + depending on the SASL mechanism (GSSAPI or PLAIN). +

+

+ See the SSL section for a description of how to configure the SSL Context Service based on the + ssl.client.auth property. +

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

Description

+

+ This Processor puts the contents of a FlowFile to a Topic in + Apache Kafka using KafkaProducer API available + with Kafka 1.0 API. The contents of the incoming FlowFile will be read using the + configured Record Reader. Each record will then be serialized using the configured + Record Writer, and this serialized form will be the content of a Kafka message. + This message is optionally assigned a key by using the <Kafka Key> Property. +

+ + +

Security Configuration

+

+ The Security Protocol property allows the user to specify the protocol for communicating + with the Kafka broker. The following sections describe each of the protocols in further detail. +

+

PLAINTEXT

+

+ This option provides an unsecured connection to the broker, with no client authentication and no encryption. + In order to use this option the broker must be configured with a listener of the form: +

+    PLAINTEXT://host.name:port
+            
+

+

SSL

+

+ This option provides an encrypted connection to the broker, with optional client authentication. In order + to use this option the broker must be configured with a listener of the form: +

+    SSL://host.name:port
+            
+ In addition, the processor must have an SSL Context Service selected. +

+

+ If the broker specifies ssl.client.auth=none, or does not specify ssl.client.auth, then the client will + not be required to present a certificate. In this case, the SSL Context Service selected may specify only + a truststore containing the public key of the certificate authority used to sign the broker's key. +

+

+ If the broker specifies ssl.client.auth=required then the client will be required to present a certificate. + In this case, the SSL Context Service must also specify a keystore containing a client key, in addition to + a truststore as described above. +

+

SASL_PLAINTEXT

+

+ This option uses SASL with a PLAINTEXT transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_PLAINTEXT://host.name:port
+            
+ In addition, the Kerberos Service Name must be specified in the processor. +

+

SASL_PLAINTEXT - GSSAPI

+

+ If the SASL mechanism is GSSAPI, then the client must provide a JAAS configuration to authenticate. The + JAAS configuration can be provided by specifying the java.security.auth.login.config system property in + NiFi's bootstrap.conf, such as: +

+    java.arg.16=-Djava.security.auth.login.config=/path/to/kafka_client_jaas.conf
+            
+

+

+ An example of the JAAS config file would be the following: +

+    KafkaClient {
+        com.sun.security.auth.module.Krb5LoginModule required
+        useKeyTab=true
+        storeKey=true
+        keyTab="/path/to/nifi.keytab"
+        serviceName="kafka"
+        principal="nifi@YOURREALM.COM";
+    };
+            
+ NOTE: The serviceName in the JAAS file must match the Kerberos Service Name in the processor. +

+

+ Alternatively, the JAAS + configuration when using GSSAPI can be provided by specifying the Kerberos Principal and Kerberos Keytab + directly in the processor properties. This will dynamically create a JAAS configuration like above, and + will take precedence over the java.security.auth.login.config system property. +

+

SASL_PLAINTEXT - PLAIN

+

+ If the SASL mechanism is PLAIN, then client must provide a JAAS configuration to authenticate, but + the JAAS configuration must use Kafka's PlainLoginModule. An example of the JAAS config file would + be the following: +

+    KafkaClient {
+      org.apache.kafka.common.security.plain.PlainLoginModule required
+      username="nifi"
+      password="nifi-password";
+    };
+            
+

+

+ NOTE: It is not recommended to use a SASL mechanism of PLAIN with SASL_PLAINTEXT, as it would transmit + the username and password unencrypted. +

+

+ NOTE: Using the PlainLoginModule will cause it be registered in the JVM's static list of Providers, making + it visible to components in other NARs that may access the providers. There is currently a known issue + where Kafka processors using the PlainLoginModule will cause HDFS processors with Keberos to no longer work. +

+

SASL_SSL

+

+ This option uses SASL with an SSL/TLS transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_SSL://host.name:port
+            
+

+

+ See the SASL_PLAINTEXT section for a description of how to provide the proper JAAS configuration + depending on the SASL mechanism (GSSAPI or PLAIN). +

+

+ See the SSL section for a description of how to configure the SSL Context Service based on the + ssl.client.auth property. +

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

Description

+

+ This Processor puts the contents of a FlowFile to a Topic in + Apache Kafka using KafkaProducer API available + with Kafka 1.0 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 FlowFile 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. +

+ + +

Security Configuration

+

+ The Security Protocol property allows the user to specify the protocol for communicating + with the Kafka broker. The following sections describe each of the protocols in further detail. +

+

PLAINTEXT

+

+ This option provides an unsecured connection to the broker, with no client authentication and no encryption. + In order to use this option the broker must be configured with a listener of the form: +

+    PLAINTEXT://host.name:port
+            
+

+

SSL

+

+ This option provides an encrypted connection to the broker, with optional client authentication. In order + to use this option the broker must be configured with a listener of the form: +

+    SSL://host.name:port
+            
+ In addition, the processor must have an SSL Context Service selected. +

+

+ If the broker specifies ssl.client.auth=none, or does not specify ssl.client.auth, then the client will + not be required to present a certificate. In this case, the SSL Context Service selected may specify only + a truststore containing the public key of the certificate authority used to sign the broker's key. +

+

+ If the broker specifies ssl.client.auth=required then the client will be required to present a certificate. + In this case, the SSL Context Service must also specify a keystore containing a client key, in addition to + a truststore as described above. +

+

SASL_PLAINTEXT

+

+ This option uses SASL with a PLAINTEXT transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_PLAINTEXT://host.name:port
+            
+ In addition, the Kerberos Service Name must be specified in the processor. +

+

SASL_PLAINTEXT - GSSAPI

+

+ If the SASL mechanism is GSSAPI, then the client must provide a JAAS configuration to authenticate. The + JAAS configuration can be provided by specifying the java.security.auth.login.config system property in + NiFi's bootstrap.conf, such as: +

+    java.arg.16=-Djava.security.auth.login.config=/path/to/kafka_client_jaas.conf
+            
+

+

+ An example of the JAAS config file would be the following: +

+    KafkaClient {
+        com.sun.security.auth.module.Krb5LoginModule required
+        useKeyTab=true
+        storeKey=true
+        keyTab="/path/to/nifi.keytab"
+        serviceName="kafka"
+        principal="nifi@YOURREALM.COM";
+    };
+            
+ NOTE: The serviceName in the JAAS file must match the Kerberos Service Name in the processor. +

+

+ Alternatively, the JAAS + configuration when using GSSAPI can be provided by specifying the Kerberos Principal and Kerberos Keytab + directly in the processor properties. This will dynamically create a JAAS configuration like above, and + will take precedence over the java.security.auth.login.config system property. +

+

SASL_PLAINTEXT - PLAIN

+

+ If the SASL mechanism is PLAIN, then client must provide a JAAS configuration to authenticate, but + the JAAS configuration must use Kafka's PlainLoginModule. An example of the JAAS config file would + be the following: +

+    KafkaClient {
+      org.apache.kafka.common.security.plain.PlainLoginModule required
+      username="nifi"
+      password="nifi-password";
+    };
+            
+

+

+ NOTE: It is not recommended to use a SASL mechanism of PLAIN with SASL_PLAINTEXT, as it would transmit + the username and password unencrypted. +

+

+ NOTE: Using the PlainLoginModule will cause it be registered in the JVM's static list of Providers, making + it visible to components in other NARs that may access the providers. There is currently a known issue + where Kafka processors using the PlainLoginModule will cause HDFS processors with Keberos to no longer work. +

+

SASL_SSL

+

+ This option uses SASL with an SSL/TLS transport layer to authenticate to the broker. In order to use this + option the broker must be configured with a listener of the form: +

+    SASL_SSL://host.name:port
+            
+

+

+ See the SASL_PLAINTEXT section for a description of how to provide the proper JAAS configuration + depending on the SASL mechanism (GSSAPI or PLAIN). +

+

+ See the SSL section for a description of how to configure the SSL Context Service based on the + ssl.client.auth property. +

+ + diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java new file mode 100644 index 0000000000..7b5a8fc5d9 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java @@ -0,0 +1,218 @@ +/* + * 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.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class ConsumeKafkaTest { + + ConsumerLease mockLease = null; + ConsumerPool mockConsumerPool = null; + + @Before + public void setup() { + mockLease = mock(ConsumerLease.class); + mockConsumerPool = mock(ConsumerPool.class); + } + + @Test + public void validateCustomValidatorSettings() throws Exception { + ConsumeKafka_1_0 consumeKafka = new ConsumeKafka_1_0(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.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_1_0 consumeKafka = new ConsumeKafka_1_0(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST); + + runner.removeProperty(ConsumeKafka_1_0.GROUP_ID); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("invalid because Group ID is required")); + } + + runner.setProperty(ConsumeKafka_1_0.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_1_0.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"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE); + when(mockLease.commit()).thenReturn(Boolean.TRUE); + + ConsumeKafka_1_0 proc = new ConsumeKafka_1_0() { + @Override + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + return mockConsumerPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(3)).continuePolling(); + verify(mockLease, times(2)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void validateGetAllMessagesPattern() throws Exception { + String groupName = "validateGetAllMessagesPattern"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE); + when(mockLease.commit()).thenReturn(Boolean.TRUE); + + ConsumeKafka_1_0 proc = new ConsumeKafka_1_0() { + @Override + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + return mockConsumerPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "(fo.*)|(ba)"); + runner.setProperty(ConsumeKafka_1_0.TOPIC_TYPE, "pattern"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(3)).continuePolling(); + verify(mockLease, times(2)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void validateGetErrorMessages() throws Exception { + String groupName = "validateGetErrorMessages"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(true, false); + when(mockLease.commit()).thenReturn(Boolean.FALSE); + + ConsumeKafka_1_0 proc = new ConsumeKafka_1_0() { + @Override + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + return mockConsumerPool; + } + }; + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(2)).continuePolling(); + verify(mockLease, times(1)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void testJaasConfiguration() throws Exception { + ConsumeKafka_1_0 consumeKafka = new ConsumeKafka_1_0(); + TestRunner runner = TestRunners.newTestRunner(consumeKafka); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + runner.setProperty(ConsumeKafka_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafka_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafka_1_0.AUTO_OFFSET_RESET, ConsumeKafka_1_0.OFFSET_EARLIEST); + + runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.KERBEROS_PRINCIPLE, "kafka"); + runner.assertValid(); + + runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM"); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File"); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties"); + runner.assertValid(); + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java new file mode 100644 index 0000000000..232af26894 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java @@ -0,0 +1,233 @@ +/* + * 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.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.regex.Pattern; + +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.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.provenance.ProvenanceReporter; +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 org.mockito.Mockito; + +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class ConsumerPoolTest { + + private Consumer consumer = null; + private ProcessSession mockSession = null; + private ProcessContext mockContext = Mockito.mock(ProcessContext.class); + private ProvenanceReporter mockReporter = null; + private ConsumerPool testPool = null; + private ConsumerPool testDemarcatedPool = null; + private ComponentLog logger = null; + + @Before + @SuppressWarnings("unchecked") + public void setup() { + consumer = mock(Consumer.class); + logger = mock(ComponentLog.class); + mockSession = mock(ProcessSession.class); + mockReporter = mock(ProvenanceReporter.class); + when(mockSession.getProvenanceReporter()).thenReturn(mockReporter); + testPool = new ConsumerPool( + 1, + null, + Collections.emptyMap(), + Collections.singletonList("nifi"), + 100L, + "utf-8", + "ssl", + "localhost", + logger, + true, + StandardCharsets.UTF_8, + null) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + testDemarcatedPool = new ConsumerPool( + 1, + "--demarcator--".getBytes(StandardCharsets.UTF_8), + Collections.emptyMap(), + Collections.singletonList("nifi"), + 100L, + "utf-8", + "ssl", + "localhost", + logger, + true, + StandardCharsets.UTF_8, + Pattern.compile(".*")) { + @Override + protected Consumer createKafkaConsumer() { + return consumer; + } + }; + } + + @Test + public void validatePoolSimpleCreateClose() throws Exception { + + when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{})); + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + } + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + } + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + } + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + } + testPool.close(); + verify(mockSession, times(0)).create(); + verify(mockSession, times(0)).commit(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(4, stats.leasesObtainedCount); + } + + @Test + @SuppressWarnings("unchecked") + public void validatePoolSimpleCreatePollClose() throws Exception { + 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); + + when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{})); + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + lease.commit(); + } + testPool.close(); + verify(mockSession, times(3)).create(); + verify(mockSession, times(1)).commit(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + } + + @Test + public void validatePoolSimpleBatchCreateClose() throws Exception { + when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{})); + for (int i = 0; i < 100; i++) { + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + for (int j = 0; j < 100; j++) { + lease.poll(); + } + } + } + testPool.close(); + verify(mockSession, times(0)).create(); + verify(mockSession, times(0)).commit(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(100, stats.leasesObtainedCount); + } + + @Test + @SuppressWarnings("unchecked") + public void validatePoolBatchCreatePollClose() throws Exception { + 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); + + when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{})); + try (final ConsumerLease lease = testDemarcatedPool.obtainConsumer(mockSession, mockContext)) { + lease.poll(); + lease.commit(); + } + testDemarcatedPool.close(); + verify(mockSession, times(1)).create(); + verify(mockSession, times(1)).commit(); + final PoolStats stats = testDemarcatedPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + } + + @Test + public void validatePoolConsumerFails() throws Exception { + + when(consumer.poll(anyLong())).thenThrow(new KafkaException("oops")); + try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) { + try { + lease.poll(); + fail(); + } catch (final KafkaException ke) { + + } + } + testPool.close(); + verify(mockSession, times(0)).create(); + verify(mockSession, times(0)).commit(); + final PoolStats stats = testPool.getPoolStats(); + assertEquals(1, stats.consumerCreatedCount); + assertEquals(1, stats.consumerClosedCount); + assertEquals(1, stats.leasesObtainedCount); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + static 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); + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java new file mode 100644 index 0000000000..d370fec1aa --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaRecord_1_0.java @@ -0,0 +1,216 @@ +/* + * 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.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser; +import org.apache.nifi.processors.kafka.pubsub.util.MockRecordWriter; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestConsumeKafkaRecord_1_0 { + + private ConsumerLease mockLease = null; + private ConsumerPool mockConsumerPool = null; + private TestRunner runner; + + @Before + public void setup() throws InitializationException { + mockLease = mock(ConsumerLease.class); + mockConsumerPool = mock(ConsumerPool.class); + + ConsumeKafkaRecord_1_0 proc = new ConsumeKafkaRecord_1_0() { + @Override + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + return mockConsumerPool; + } + }; + + runner = TestRunners.newTestRunner(proc); + runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234"); + + final String readerId = "record-reader"; + final MockRecordParser readerService = new MockRecordParser(); + readerService.addSchemaField("name", RecordFieldType.STRING); + readerService.addSchemaField("age", RecordFieldType.INT); + runner.addControllerService(readerId, readerService); + runner.enableControllerService(readerService); + + final String writerId = "record-writer"; + final RecordSetWriterFactory writerService = new MockRecordWriter("name, age"); + runner.addControllerService(writerId, writerService); + runner.enableControllerService(writerService); + + runner.setProperty(ConsumeKafkaRecord_1_0.RECORD_READER, readerId); + runner.setProperty(ConsumeKafkaRecord_1_0.RECORD_WRITER, writerId); + } + + @Test + public void validateCustomValidatorSettings() throws Exception { + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.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 { + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST); + + runner.removeProperty(ConsumeKafkaRecord_1_0.GROUP_ID); + try { + runner.assertValid(); + fail(); + } catch (AssertionError e) { + assertTrue(e.getMessage().contains("invalid because Group ID is required")); + } + + runner.setProperty(ConsumeKafkaRecord_1_0.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(ConsumeKafkaRecord_1_0.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"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE); + when(mockLease.commit()).thenReturn(Boolean.TRUE); + + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(3)).continuePolling(); + verify(mockLease, times(2)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void validateGetAllMessagesPattern() throws Exception { + String groupName = "validateGetAllMessagesPattern"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE); + when(mockLease.commit()).thenReturn(Boolean.TRUE); + + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "(fo.*)|(ba)"); + runner.setProperty(ConsumeKafkaRecord_1_0.TOPIC_TYPE, "pattern"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(3)).continuePolling(); + verify(mockLease, times(2)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void validateGetErrorMessages() throws Exception { + String groupName = "validateGetErrorMessages"; + + when(mockConsumerPool.obtainConsumer(anyObject(), anyObject())).thenReturn(mockLease); + when(mockLease.continuePolling()).thenReturn(true, false); + when(mockLease.commit()).thenReturn(Boolean.FALSE); + + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo,bar"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, groupName); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST); + runner.run(1, false); + + verify(mockConsumerPool, times(1)).obtainConsumer(anyObject(), anyObject()); + verify(mockLease, times(2)).continuePolling(); + verify(mockLease, times(1)).poll(); + verify(mockLease, times(1)).commit(); + verify(mockLease, times(1)).close(); + verifyNoMoreInteractions(mockConsumerPool); + verifyNoMoreInteractions(mockLease); + } + + @Test + public void testJaasConfiguration() throws Exception { + runner.setProperty(ConsumeKafkaRecord_1_0.TOPICS, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.GROUP_ID, "foo"); + runner.setProperty(ConsumeKafkaRecord_1_0.AUTO_OFFSET_RESET, ConsumeKafkaRecord_1_0.OFFSET_EARLIEST); + + runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.KERBEROS_PRINCIPLE, "kafka"); + runner.assertValid(); + + runner.setProperty(KafkaProcessorUtils.USER_PRINCIPAL, "nifi@APACHE.COM"); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "not.A.File"); + runner.assertNotValid(); + + runner.setProperty(KafkaProcessorUtils.USER_KEYTAB, "src/test/resources/server.properties"); + runner.assertValid(); + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java new file mode 100644 index 0000000000..78c03ec435 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java @@ -0,0 +1,88 @@ +/* + * 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.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeoutException; + +import org.apache.nifi.util.MockComponentLog; +import org.apache.nifi.util.MockFlowFile; +import org.junit.Assert; +import org.junit.Test; + +public class TestInFlightMessageTracker { + + @Test(timeout = 5000L) + public void testAwaitCompletionWhenComplete() throws InterruptedException, TimeoutException { + final MockFlowFile flowFile = new MockFlowFile(1L); + + final InFlightMessageTracker tracker = new InFlightMessageTracker(new MockComponentLog("1", "unit-test")); + tracker.incrementSentCount(flowFile); + + verifyNotComplete(tracker); + + tracker.incrementSentCount(flowFile); + verifyNotComplete(tracker); + + tracker.incrementAcknowledgedCount(flowFile); + verifyNotComplete(tracker); + + tracker.incrementAcknowledgedCount(flowFile); + tracker.awaitCompletion(1L); + } + + @Test(timeout = 5000L) + public void testAwaitCompletionWhileWaiting() throws InterruptedException, ExecutionException { + final MockFlowFile flowFile = new MockFlowFile(1L); + + final InFlightMessageTracker tracker = new InFlightMessageTracker(new MockComponentLog("1", "unit-test")); + tracker.incrementSentCount(flowFile); + + verifyNotComplete(tracker); + + tracker.incrementSentCount(flowFile); + verifyNotComplete(tracker); + + final ExecutorService exec = Executors.newFixedThreadPool(1); + final Future future = exec.submit(() -> { + try { + tracker.awaitCompletion(10000L); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + tracker.incrementAcknowledgedCount(flowFile); + tracker.incrementAcknowledgedCount(flowFile); + + future.get(); + } + + private void verifyNotComplete(final InFlightMessageTracker tracker) throws InterruptedException { + try { + tracker.awaitCompletion(10L); + Assert.fail("Expected timeout"); + } catch (final TimeoutException te) { + // expected + } + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java new file mode 100644 index 0000000000..6a0c7cea67 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java @@ -0,0 +1,250 @@ +/* + * 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.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestPublishKafka { + private static final String TOPIC_NAME = "unit-test"; + + private PublisherPool mockPool; + private PublisherLease mockLease; + private TestRunner runner; + + @Before + public void setup() { + mockPool = mock(PublisherPool.class); + mockLease = mock(PublisherLease.class); + + when(mockPool.obtainPublisher()).thenReturn(mockLease); + + runner = TestRunners.newTestRunner(new PublishKafka_1_0() { + @Override + protected PublisherPool createPublisherPool(final ProcessContext context) { + return mockPool; + } + }); + + runner.setProperty(PublishKafka_1_0.TOPIC, TOPIC_NAME); + runner.setProperty(PublishKafka_1_0.DELIVERY_GUARANTEE, PublishKafka_1_0.DELIVERY_REPLICATED); + } + + @Test + public void testSingleSuccess() throws IOException { + final MockFlowFile flowFile = runner.enqueue("hello world"); + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFile, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_1_0.REL_SUCCESS, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleSuccess() throws IOException { + final Set flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFiles, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_1_0.REL_SUCCESS, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testSingleFailure() throws IOException { + final MockFlowFile flowFile = runner.enqueue("hello world"); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_1_0.REL_FAILURE, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleFailures() throws IOException { + final Set flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFiles)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_1_0.REL_FAILURE, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleMessagesPerFlowFile() throws IOException { + final List flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + final Map msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap()); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_1_0.REL_SUCCESS, 2); + + verify(mockLease, times(2)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + + runner.assertAllFlowFilesContainAttribute("msg.count"); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_1_0.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("10")) + .count()); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_1_0.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("20")) + .count()); + } + + + @Test + public void testSomeSuccessSomeFailure() throws IOException { + final List flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + final Map msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final Map failureMap = new HashMap<>(); + failureMap.put(flowFiles.get(2), new RuntimeException("Intentional Unit Test Exception")); + failureMap.put(flowFiles.get(3), new RuntimeException("Intentional Unit Test Exception")); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles.subList(0, 2)), failureMap); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertTransferCount(PublishKafka_1_0.REL_SUCCESS, 0); + runner.assertTransferCount(PublishKafka_1_0.REL_FAILURE, 4); + + verify(mockLease, times(4)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + + assertTrue(runner.getFlowFilesForRelationship(PublishKafka_1_0.REL_FAILURE).stream() + .noneMatch(ff -> ff.getAttribute("msg.count") != null)); + } + + + private PublishResult createAllSuccessPublishResult(final FlowFile successfulFlowFile, final int msgCount) { + return createAllSuccessPublishResult(Collections.singleton(successfulFlowFile), msgCount); + } + + private PublishResult createAllSuccessPublishResult(final Set successfulFlowFiles, final int msgCountPerFlowFile) { + final Map msgCounts = new HashMap<>(); + for (final FlowFile ff : successfulFlowFiles) { + msgCounts.put(ff, msgCountPerFlowFile); + } + return createPublishResult(msgCounts, successfulFlowFiles, Collections.emptyMap()); + } + + private PublishResult createFailurePublishResult(final FlowFile failure) { + return createFailurePublishResult(Collections.singleton(failure)); + } + + private PublishResult createFailurePublishResult(final Set failures) { + final Map failureMap = failures.stream().collect(Collectors.toMap(ff -> ff, ff -> new RuntimeException("Intentional Unit Test Exception"))); + return createPublishResult(Collections.emptyMap(), Collections.emptySet(), failureMap); + } + + private PublishResult createPublishResult(final Map msgCounts, final Set successFlowFiles, final Map failures) { + // sanity check. + for (final FlowFile success : successFlowFiles) { + if (failures.containsKey(success)) { + throw new IllegalArgumentException("Found same FlowFile in both 'success' and 'failures' collections: " + success); + } + } + + return new PublishResult() { + @Override + public boolean isFailure() { + return !failures.isEmpty(); + } + + @Override + public int getSuccessfulMessageCount(FlowFile flowFile) { + Integer count = msgCounts.get(flowFile); + return count == null ? 0 : count.intValue(); + } + + @Override + public Exception getReasonForFailure(FlowFile flowFile) { + return failures.get(flowFile); + } + }; + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_1_0.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_1_0.java new file mode 100644 index 0000000000..45439cc126 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaRecord_1_0.java @@ -0,0 +1,300 @@ +/* + * 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.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processors.kafka.pubsub.util.MockRecordParser; +import org.apache.nifi.processors.kafka.pubsub.util.MockRecordWriter; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordSet; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestPublishKafkaRecord_1_0 { + + private static final String TOPIC_NAME = "unit-test"; + + private PublisherPool mockPool; + private PublisherLease mockLease; + private TestRunner runner; + + @Before + public void setup() throws InitializationException, IOException { + mockPool = mock(PublisherPool.class); + mockLease = mock(PublisherLease.class); + Mockito.doCallRealMethod().when(mockLease).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), + any(RecordSchema.class), any(String.class), any(String.class)); + + when(mockPool.obtainPublisher()).thenReturn(mockLease); + + runner = TestRunners.newTestRunner(new PublishKafkaRecord_1_0() { + @Override + protected PublisherPool createPublisherPool(final ProcessContext context) { + return mockPool; + } + }); + + runner.setProperty(PublishKafkaRecord_1_0.TOPIC, TOPIC_NAME); + + final String readerId = "record-reader"; + final MockRecordParser readerService = new MockRecordParser(); + readerService.addSchemaField("name", RecordFieldType.STRING); + readerService.addSchemaField("age", RecordFieldType.INT); + runner.addControllerService(readerId, readerService); + runner.enableControllerService(readerService); + + final String writerId = "record-writer"; + final RecordSetWriterFactory writerService = new MockRecordWriter("name, age"); + runner.addControllerService(writerId, writerService); + runner.enableControllerService(writerService); + + runner.setProperty(PublishKafkaRecord_1_0.RECORD_READER, readerId); + runner.setProperty(PublishKafkaRecord_1_0.RECORD_WRITER, writerId); + runner.setProperty(PublishKafka_1_0.DELIVERY_GUARANTEE, PublishKafka_1_0.DELIVERY_REPLICATED); + } + + @Test + public void testSingleSuccess() throws IOException { + final MockFlowFile flowFile = runner.enqueue("John Doe, 48"); + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFile, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_SUCCESS, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleSuccess() throws IOException { + final Set flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFiles, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_SUCCESS, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testSingleFailure() throws IOException { + final MockFlowFile flowFile = runner.enqueue("John Doe, 48"); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_FAILURE, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleFailures() throws IOException { + final Set flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFiles)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_FAILURE, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleMessagesPerFlowFile() throws IOException { + final List flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("John Doe, 48\nJane Doe, 47")); + flowFiles.add(runner.enqueue("John Doe, 48\nJane Doe, 29")); + + final Map msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap()); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_SUCCESS, 2); + + verify(mockLease, times(2)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(4)).publish(any(FlowFile.class), eq(null), any(byte[].class), eq(TOPIC_NAME), any(InFlightMessageTracker.class)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + + runner.assertAllFlowFilesContainAttribute("msg.count"); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafkaRecord_1_0.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("10")) + .count()); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafkaRecord_1_0.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("20")) + .count()); + } + + @Test + public void testNoRecordsInFlowFile() throws IOException { + final List flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue(new byte[0])); + + final Map msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 0); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap()); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafkaRecord_1_0.REL_SUCCESS, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + + final MockFlowFile mff = runner.getFlowFilesForRelationship(PublishKafkaRecord_1_0.REL_SUCCESS).get(0); + mff.assertAttributeEquals("msg.count", "0"); + } + + + @Test + public void testSomeSuccessSomeFailure() throws IOException { + final List flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + flowFiles.add(runner.enqueue("John Doe, 48")); + + final Map msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final Map failureMap = new HashMap<>(); + failureMap.put(flowFiles.get(2), new RuntimeException("Intentional Unit Test Exception")); + failureMap.put(flowFiles.get(3), new RuntimeException("Intentional Unit Test Exception")); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles.subList(0, 2)), failureMap); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertTransferCount(PublishKafkaRecord_1_0.REL_SUCCESS, 0); + runner.assertTransferCount(PublishKafkaRecord_1_0.REL_FAILURE, 4); + + verify(mockLease, times(4)).publish(any(FlowFile.class), any(RecordSet.class), any(RecordSetWriterFactory.class), any(RecordSchema.class), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + + assertTrue(runner.getFlowFilesForRelationship(PublishKafkaRecord_1_0.REL_FAILURE).stream() + .noneMatch(ff -> ff.getAttribute("msg.count") != null)); + } + + + private PublishResult createAllSuccessPublishResult(final FlowFile successfulFlowFile, final int msgCount) { + return createAllSuccessPublishResult(Collections.singleton(successfulFlowFile), msgCount); + } + + private PublishResult createAllSuccessPublishResult(final Set successfulFlowFiles, final int msgCountPerFlowFile) { + final Map msgCounts = new HashMap<>(); + for (final FlowFile ff : successfulFlowFiles) { + msgCounts.put(ff, msgCountPerFlowFile); + } + return createPublishResult(msgCounts, successfulFlowFiles, Collections.emptyMap()); + } + + private PublishResult createFailurePublishResult(final FlowFile failure) { + return createFailurePublishResult(Collections.singleton(failure)); + } + + private PublishResult createFailurePublishResult(final Set failures) { + final Map failureMap = failures.stream().collect(Collectors.toMap(ff -> ff, ff -> new RuntimeException("Intentional Unit Test Exception"))); + return createPublishResult(Collections.emptyMap(), Collections.emptySet(), failureMap); + } + + private PublishResult createPublishResult(final Map msgCounts, final Set successFlowFiles, final Map failures) { + // sanity check. + for (final FlowFile success : successFlowFiles) { + if (failures.containsKey(success)) { + throw new IllegalArgumentException("Found same FlowFile in both 'success' and 'failures' collections: " + success); + } + } + + return new PublishResult() { + + @Override + public int getSuccessfulMessageCount(FlowFile flowFile) { + Integer count = msgCounts.get(flowFile); + return count == null ? 0 : count.intValue(); + } + + @Override + public Exception getReasonForFailure(FlowFile flowFile) { + return failures.get(flowFile); + } + + @Override + public boolean isFailure() { + return !failures.isEmpty(); + } + }; + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java new file mode 100644 index 0000000000..54c122231e --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java @@ -0,0 +1,190 @@ +/* + * 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.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.util.MockFlowFile; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + + +public class TestPublisherLease { + private ComponentLog logger; + private Producer producer; + + @Before + @SuppressWarnings("unchecked") + public void setup() { + logger = Mockito.mock(ComponentLog.class); + producer = Mockito.mock(Producer.class); + } + + @Test + public void testPoisonOnException() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger, true, null, StandardCharsets.UTF_8) { + @Override + public void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = null; + + final InputStream failureInputStream = new InputStream() { + @Override + public int read() throws IOException { + throw new IOException("Intentional Unit Test Exception"); + } + }; + + try { + lease.publish(flowFile, failureInputStream, messageKey, demarcatorBytes, topic); + Assert.fail("Expected IOException"); + } catch (final IOException ioe) { + // expected + } + + assertEquals(1, poisonCount.get()); + + final PublishResult result = lease.complete(); + assertTrue(result.isFailure()); + } + + @Test + @SuppressWarnings("unchecked") + public void testPoisonOnFailure() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger, true, null, StandardCharsets.UTF_8) { + @Override + public void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = null; + + doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final Callback callback = invocation.getArgumentAt(1, Callback.class); + callback.onCompletion(null, new RuntimeException("Unit Test Intentional Exception")); + return null; + } + }).when(producer).send(any(ProducerRecord.class), any(Callback.class)); + + lease.publish(flowFile, new ByteArrayInputStream(new byte[1]), messageKey, demarcatorBytes, topic); + + assertEquals(1, poisonCount.get()); + + final PublishResult result = lease.complete(); + assertTrue(result.isFailure()); + } + + @Test + @SuppressWarnings("unchecked") + public void testAllDelimitedMessagesSent() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger, true, null, StandardCharsets.UTF_8) { + @Override + protected void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final AtomicInteger correctMessages = new AtomicInteger(0); + final AtomicInteger incorrectMessages = new AtomicInteger(0); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final ProducerRecord record = invocation.getArgumentAt(0, ProducerRecord.class); + final byte[] value = record.value(); + final String valueString = new String(value, StandardCharsets.UTF_8); + if ("1234567890".equals(valueString)) { + correctMessages.incrementAndGet(); + } else { + incorrectMessages.incrementAndGet(); + } + + return null; + } + }).when(producer).send(any(ProducerRecord.class), any(Callback.class)); + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = "\n".getBytes(StandardCharsets.UTF_8); + + final byte[] flowFileContent = "1234567890\n1234567890\n1234567890\n\n\n\n1234567890\n\n\n1234567890\n\n\n\n".getBytes(StandardCharsets.UTF_8); + lease.publish(flowFile, new ByteArrayInputStream(flowFileContent), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent2 = new byte[0]; + lease.publish(new MockFlowFile(2L), new ByteArrayInputStream(flowFileContent2), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent3 = "1234567890\n1234567890".getBytes(StandardCharsets.UTF_8); // no trailing new line + lease.publish(new MockFlowFile(3L), new ByteArrayInputStream(flowFileContent3), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent4 = "\n\n\n".getBytes(StandardCharsets.UTF_8); + lease.publish(new MockFlowFile(4L), new ByteArrayInputStream(flowFileContent4), messageKey, demarcatorBytes, topic); + + assertEquals(0, poisonCount.get()); + + verify(producer, times(0)).flush(); + + final PublishResult result = lease.complete(); + assertTrue(result.isFailure()); + + assertEquals(7, correctMessages.get()); + assertEquals(0, incorrectMessages.get()); + + verify(producer, times(1)).flush(); + } +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java new file mode 100644 index 0000000000..afaf8414c0 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java @@ -0,0 +1,69 @@ +/* + * 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 java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.logging.ComponentLog; +import org.junit.Test; +import org.mockito.Mockito; + + +public class TestPublisherPool { + + @Test + public void testLeaseCloseReturnsToPool() { + final Map kafkaProperties = new HashMap<>(); + kafkaProperties.put("bootstrap.servers", "localhost:1111"); + kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName()); + kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName()); + + final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8); + assertEquals(0, pool.available()); + + final PublisherLease lease = pool.obtainPublisher(); + assertEquals(0, pool.available()); + + lease.close(); + assertEquals(1, pool.available()); + } + + @Test + public void testPoisonedLeaseNotReturnedToPool() { + final Map kafkaProperties = new HashMap<>(); + kafkaProperties.put("bootstrap.servers", "localhost:1111"); + kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName()); + kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName()); + + final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L, false, null, StandardCharsets.UTF_8); + assertEquals(0, pool.available()); + + final PublisherLease lease = pool.obtainPublisher(); + assertEquals(0, pool.available()); + + lease.poison(); + lease.close(); + assertEquals(0, pool.available()); + } + +} diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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-1-0-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-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordParser.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordParser.java new file mode 100644 index 0000000000..21527be89f --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordParser.java @@ -0,0 +1,105 @@ +/* + * 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.util; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaValidationException; +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.MapRecord; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; + +public class MockRecordParser extends AbstractControllerService implements RecordReaderFactory { + private final List records = new ArrayList<>(); + private final List fields = new ArrayList<>(); + private final int failAfterN; + + public MockRecordParser() { + this(-1); + } + + public MockRecordParser(final int failAfterN) { + this.failAfterN = failAfterN; + } + + + public void addSchemaField(final String fieldName, final RecordFieldType type) { + fields.add(new RecordField(fieldName, type.getDataType())); + } + + public void addRecord(Object... values) { + records.add(values); + } + + @Override + public RecordReader createRecordReader(Map variables, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { + final BufferedReader reader = new BufferedReader(new InputStreamReader(in)); + + return new RecordReader() { + private int recordCount = 0; + + @Override + public void close() throws IOException { + } + + @Override + public Record nextRecord(boolean coerceTypes, boolean dropUnknown) throws IOException, MalformedRecordException, SchemaValidationException { + if (failAfterN >= recordCount) { + throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read"); + } + final String line = reader.readLine(); + if (line == null) { + return null; + } + + recordCount++; + + final String[] values = line.split(","); + final Map valueMap = new HashMap<>(); + int i = 0; + for (final RecordField field : fields) { + final String fieldName = field.getFieldName(); + valueMap.put(fieldName, values[i++].trim()); + } + + return new MapRecord(new SimpleRecordSchema(fields), valueMap); + } + + @Override + public RecordSchema getSchema() { + return new SimpleRecordSchema(fields); + } + }; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordWriter.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordWriter.java new file mode 100644 index 0000000000..90a909d9b6 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/util/MockRecordWriter.java @@ -0,0 +1,128 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.Map; + +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordSet; + +public class MockRecordWriter extends AbstractControllerService implements RecordSetWriterFactory { + private final String header; + private final int failAfterN; + private final boolean quoteValues; + + public MockRecordWriter(final String header) { + this(header, true, -1); + } + + public MockRecordWriter(final String header, final boolean quoteValues) { + this(header, quoteValues, -1); + } + + public MockRecordWriter(final String header, final boolean quoteValues, final int failAfterN) { + this.header = header; + this.quoteValues = quoteValues; + this.failAfterN = failAfterN; + } + + @Override + public RecordSchema getSchema(Map variables, RecordSchema readSchema) throws SchemaNotFoundException, IOException { + return null; + } + + @Override + public RecordSetWriter createWriter(final ComponentLog logger, final RecordSchema schema, final OutputStream out) { + return new RecordSetWriter() { + + @Override + public void flush() throws IOException { + out.flush(); + } + + @Override + public WriteResult write(final RecordSet rs) throws IOException { + out.write(header.getBytes()); + out.write("\n".getBytes()); + + int recordCount = 0; + final int numCols = rs.getSchema().getFieldCount(); + Record record = null; + while ((record = rs.next()) != null) { + if (++recordCount > failAfterN && failAfterN > -1) { + throw new IOException("Unit Test intentionally throwing IOException after " + failAfterN + " records were written"); + } + + int i = 0; + for (final String fieldName : record.getSchema().getFieldNames()) { + final String val = record.getAsString(fieldName); + if (quoteValues) { + out.write("\"".getBytes()); + if (val != null) { + out.write(val.getBytes()); + } + out.write("\"".getBytes()); + } else if (val != null) { + out.write(val.getBytes()); + } + + if (i++ < numCols - 1) { + out.write(",".getBytes()); + } + } + out.write("\n".getBytes()); + } + + return WriteResult.of(recordCount, Collections.emptyMap()); + } + + @Override + public String getMimeType() { + return "text/plain"; + } + + @Override + public WriteResult write(Record record) throws IOException { + return null; + } + + @Override + public void close() throws IOException { + } + + @Override + public void beginRecordSet() throws IOException { + } + + @Override + public WriteResult finishRecordSet() throws IOException { + return null; + } + }; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java new file mode 100644 index 0000000000..a720b118ae --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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("Embedded 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("Embedded 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-1-0-processors/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/log4j.properties new file mode 100644 index 0000000000..57cd63f139 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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-1-0-processors/src/test/resources/server.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/server.properties new file mode 100644 index 0000000000..2ecb1b20ba --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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-1-0-processors/src/test/resources/zookeeper.properties b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-processors/src/test/resources/zookeeper.properties new file mode 100644 index 0000000000..f5c257efeb --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-1-0-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/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml index 11f9ece182..e0edc7891a 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/pom.xml @@ -27,6 +27,7 @@ 0.9.0.1 0.10.2.1 0.11.0.1 + 1.0.0 @@ -34,10 +35,12 @@ nifi-kafka-0-9-processors nifi-kafka-0-10-processors nifi-kafka-0-11-processors + nifi-kafka-1-0-processors nifi-kafka-0-8-nar nifi-kafka-0-9-nar nifi-kafka-0-10-nar nifi-kafka-0-11-nar + nifi-kafka-1-0-nar @@ -61,6 +64,11 @@ nifi-kafka-0-11-processors 1.5.0-SNAPSHOT + + org.apache.nifi + nifi-kafka-1-0-processors + 1.5.0-SNAPSHOT + diff --git a/pom.xml b/pom.xml index 3f2d73635e..a5f408cef1 100644 --- a/pom.xml +++ b/pom.xml @@ -1084,6 +1084,12 @@ 1.5.0-SNAPSHOT nar + + org.apache.nifi + nifi-kafka-1-0-nar + 1.5.0-SNAPSHOT + nar + org.apache.nifi nifi-rethinkdb-nar