You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2017/09/23 02:56:38 UTC

[5/5] nifi git commit: NIFI-4201: This closes #2024. Implementation of processors for interacting with Kafka 0.11

NIFI-4201: This closes #2024. Implementation of processors for interacting with Kafka 0.11

Signed-off-by: joewitt <jo...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/3fb704c5
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/3fb704c5
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/3fb704c5

Branch: refs/heads/master
Commit: 3fb704c58f44f106779ac536bb6d5802c829f626
Parents: a29348f
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Jul 20 14:01:26 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Fri Sep 22 22:08:19 2017 -0400

----------------------------------------------------------------------
 nifi-assembly/pom.xml                           |   5 +
 .../nifi-kafka-0-11-nar/pom.xml                 |  40 ++
 .../src/main/resources/META-INF/LICENSE         | 233 +++++++
 .../src/main/resources/META-INF/NOTICE          |  83 +++
 .../nifi-kafka-0-11-processors/pom.xml          |  92 +++
 .../kafka/pubsub/ConsumeKafkaRecord_0_11.java   | 395 +++++++++++
 .../kafka/pubsub/ConsumeKafka_0_11.java         | 389 +++++++++++
 .../processors/kafka/pubsub/ConsumerLease.java  | 691 +++++++++++++++++++
 .../processors/kafka/pubsub/ConsumerPool.java   | 372 ++++++++++
 .../kafka/pubsub/InFlightMessageTracker.java    | 175 +++++
 .../kafka/pubsub/KafkaProcessorUtils.java       | 332 +++++++++
 .../processors/kafka/pubsub/Partitioners.java   |  62 ++
 .../kafka/pubsub/PublishKafkaRecord_0_11.java   | 446 ++++++++++++
 .../kafka/pubsub/PublishKafka_0_11.java         | 461 +++++++++++++
 .../processors/kafka/pubsub/PublishResult.java  |  46 ++
 .../processors/kafka/pubsub/PublisherLease.java | 264 +++++++
 .../processors/kafka/pubsub/PublisherPool.java  | 115 +++
 .../org.apache.nifi.processor.Processor         |  18 +
 .../additionalDetails.html                      | 143 ++++
 .../additionalDetails.html                      | 143 ++++
 .../additionalDetails.html                      | 144 ++++
 .../additionalDetails.html                      | 156 +++++
 .../kafka/pubsub/ConsumeKafkaTest.java          | 218 ++++++
 .../kafka/pubsub/ConsumerPoolTest.java          | 233 +++++++
 .../pubsub/TestConsumeKafkaRecord_0_11.java     | 216 ++++++
 .../pubsub/TestInFlightMessageTracker.java      |  88 +++
 .../kafka/pubsub/TestPublishKafka.java          | 250 +++++++
 .../pubsub/TestPublishKafkaRecord_0_11.java     | 300 ++++++++
 .../kafka/pubsub/TestPublisherLease.java        | 190 +++++
 .../kafka/pubsub/TestPublisherPool.java         |  69 ++
 .../nifi/processors/kafka/pubsub/TestUtils.java |  45 ++
 .../kafka/pubsub/util/MockRecordParser.java     | 105 +++
 .../kafka/pubsub/util/MockRecordWriter.java     | 128 ++++
 .../processors/kafka/test/EmbeddedKafka.java    | 226 ++++++
 .../src/test/resources/log4j.properties         |  21 +
 .../src/test/resources/server.properties        | 121 ++++
 .../src/test/resources/zookeeper.properties     |  20 +
 nifi-nar-bundles/nifi-kafka-bundle/pom.xml      |   8 +
 pom.xml                                         |   6 +
 39 files changed, 7049 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index f348ff9..8cf5b19 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -188,6 +188,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-0-11-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-confluent-platform-nar</artifactId>
             <type>nar</type>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
new file mode 100644
index 0000000..ae4a7ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/pom.xml
@@ -0,0 +1,40 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-kafka-bundle</artifactId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-kafka-0-11-nar</artifactId>
+    <packaging>nar</packaging>
+    <description>NiFi NAR for interacting with Apache Kafka</description>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-0-11-processors</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..43a2a3b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-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.
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..03af5fd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-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.
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
new file mode 100644
index 0000000..79ee594
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/pom.xml
@@ -0,0 +1,92 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-kafka-bundle</artifactId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>nifi-kafka-0-11-processors</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <version>${kafka11.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka_2.11</artifactId>
+            <version>${kafka11.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <!-- Transitive dependencies excluded because they are located
+                in a legacy Maven repository, which Maven 3 doesn't support. -->
+                <exclusion>
+                    <groupId>javax.jms</groupId>
+                    <artifactId>jms</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.sun.jdmk</groupId>
+                    <artifactId>jmxtools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.sun.jmx</groupId>
+                    <artifactId>jmxri</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.java
new file mode 100644
index 0000000..5a450c4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_0_11.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 0.11.x Consumer API. "
+    + "The complementary NiFi processor for sending messages is PublishKafkaRecord_0_11. 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 <Headers to Add as Attributes> property.")
+@Tags({"Kafka", "Get", "Record", "csv", "avro", "json", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.11.x"})
+@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_0_11.class, PublishKafka_0_11.class, PublishKafkaRecord_0_11.class})
+public class ConsumeKafkaRecord_0_11 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<PropertyDescriptor> DESCRIPTORS;
+    static final Set<Relationship> RELATIONSHIPS;
+
+    private volatile ConsumerPool consumerPool = null;
+    private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
+
+    static {
+        List<PropertyDescriptor> 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<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_PARSE_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> 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<ValidationResult> 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<String, Object> 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_0_11.TOPICS).evaluateAttributeExpressions().getValue();
+        final String topicType = context.getProperty(ConsumeKafkaRecord_0_11.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
+        final List<String> 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);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/3fb704c5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
new file mode 100644
index 0000000..1ba23f0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-11-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_11.java
@@ -0,0 +1,389 @@
+/*
+ * 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 0.11.x Consumer API. "
+        + " Please note there are cases where the publisher can get into an indefinite stuck state.  We are closely monitoring"
+        + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can.  In the meantime"
+        + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on. "
+        + "The complementary NiFi processor for sending messages is PublishKafka_0_11.")
+@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.11.x"})
+@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_0_11 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<PropertyDescriptor> DESCRIPTORS;
+    static final Set<Relationship> RELATIONSHIPS;
+
+    private volatile ConsumerPool consumerPool = null;
+    private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
+
+    static {
+        List<PropertyDescriptor> 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<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> 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<ValidationResult> 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_0_11.MESSAGE_DEMARCATOR).isSet()
+                ? context.getProperty(ConsumeKafka_0_11.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
+                : null;
+        final Map<String, Object> 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_0_11.TOPICS).evaluateAttributeExpressions().getValue();
+        final String topicType = context.getProperty(ConsumeKafka_0_11.TOPIC_TYPE).evaluateAttributeExpressions().getValue();
+        final List<String> 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);
+            }
+        }
+    }
+}