You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jp...@apache.org on 2016/12/15 18:51:40 UTC

[01/20] nifi git commit: NIFI-2687: RPG Port ID was used instead of RPG ID

Repository: nifi
Updated Branches:
  refs/heads/support/nifi-1.0.x fc2941553 -> 5d81cabdc


NIFI-2687: RPG Port ID was used instead of RPG ID

It caused "Error: Unable to find remote process group with id 'XXXX'" error with modifying the transmitting toggle switch of an Input or Output port.

This closes #962


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

Branch: refs/heads/support/nifi-1.0.x
Commit: d5f94c44305fa4b5d509074e0f2770eb784a4f1f
Parents: fc29415
Author: Koji Kawamura <ij...@apache.org>
Authored: Mon Aug 29 14:12:46 2016 +0900
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:20:15 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/web/api/RemoteProcessGroupResource.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/d5f94c44/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
index 0f91aca..04f6d19 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
@@ -299,7 +299,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
                     final RemoteProcessGroupPortDTO remoteProcessGroupPort = remoteProcessGroupPortEntity.getRemoteProcessGroupPort();
 
                     // update the specified remote process group
-                    final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupInputPort(revision, remoteProcessGroupPort.getId(), remoteProcessGroupPort);
+                    final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupInputPort(revision, id, remoteProcessGroupPort);
 
                     // get the updated revision
                     final RevisionDTO updatedRevision = controllerResponse.getRevision();
@@ -401,7 +401,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
                     final RemoteProcessGroupPortDTO remoteProcessGroupPort = remoteProcessGroupPortEntity.getRemoteProcessGroupPort();
 
                     // update the specified remote process group
-                    final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupOutputPort(revision, remoteProcessGroupPort.getId(), remoteProcessGroupPort);
+                    final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupOutputPort(revision, id, remoteProcessGroupPort);
 
                     // get the updated revision
                     final RevisionDTO updatedRevision = controllerResponse.getRevision();


[09/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
new file mode 100644
index 0000000..c7d1a60
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
@@ -0,0 +1,262 @@
+/*
+ * 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.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.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_0_10() {
+            @Override
+            protected PublisherPool createPublisherPool(final ProcessContext context) {
+                return mockPool;
+            }
+        });
+
+        runner.setProperty(PublishKafka_0_10.TOPIC, TOPIC_NAME);
+    }
+
+    @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_0_10.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<FlowFile> 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_0_10.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_0_10.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<FlowFile> 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_0_10.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<FlowFile> flowFiles = new ArrayList<>();
+        flowFiles.add(runner.enqueue("hello world"));
+        flowFiles.add(runner.enqueue("hello world"));
+
+        final Map<FlowFile, Integer> 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_0_10.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_0_10.REL_SUCCESS).stream()
+            .filter(ff -> ff.getAttribute("msg.count").equals("10"))
+            .count());
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
+            .filter(ff -> ff.getAttribute("msg.count").equals("20"))
+            .count());
+    }
+
+
+    @Test
+    public void testSomeSuccessSomeFailure() throws IOException {
+        final List<FlowFile> 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<FlowFile, Integer> msgCounts = new HashMap<>();
+        msgCounts.put(flowFiles.get(0), 10);
+        msgCounts.put(flowFiles.get(1), 20);
+
+        final Map<FlowFile, Exception> 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_0_10.REL_SUCCESS, 2);
+        runner.assertTransferCount(PublishKafka_0_10.REL_FAILURE, 2);
+
+        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();
+
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
+            .filter(ff -> "10".equals(ff.getAttribute("msg.count")))
+            .count());
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
+            .filter(ff -> "20".equals(ff.getAttribute("msg.count")))
+            .count());
+
+        assertTrue(runner.getFlowFilesForRelationship(PublishKafka_0_10.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<FlowFile> successfulFlowFiles, final int msgCountPerFlowFile) {
+        final Map<FlowFile, Integer> 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<FlowFile> failures) {
+        final Map<FlowFile, Exception> 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<FlowFile, Integer> msgCounts, final Set<FlowFile> successFlowFiles, final Map<FlowFile, Exception> 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 Collection<FlowFile> getSuccessfulFlowFiles() {
+                return successFlowFiles;
+            }
+
+            @Override
+            public Collection<FlowFile> getFailedFlowFiles() {
+                return failures.keySet();
+            }
+
+            @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);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
new file mode 100644
index 0000000..c2d143c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
@@ -0,0 +1,194 @@
+/*
+ * 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.assertFalse;
+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<byte[], byte[]> 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) {
+            @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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testPoisonOnFailure() throws IOException {
+        final AtomicInteger poisonCount = new AtomicInteger(0);
+
+        final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
+            @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<Object>() {
+            @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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAllDelimitedMessagesSent() throws IOException {
+        final AtomicInteger poisonCount = new AtomicInteger(0);
+
+        final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger) {
+            @Override
+            protected void poison() {
+                poisonCount.incrementAndGet();
+                super.poison();
+            }
+        };
+
+        final AtomicInteger correctMessages = new AtomicInteger(0);
+        final AtomicInteger incorrectMessages = new AtomicInteger(0);
+        doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                final ProducerRecord<byte[], byte[]> 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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+
+        assertEquals(7, correctMessages.get());
+        assertEquals(0, incorrectMessages.get());
+
+        verify(producer, times(1)).flush();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
new file mode 100644
index 0000000..7c70194
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+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<String, Object> 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);
+        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<String, Object> 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);
+        assertEquals(0, pool.available());
+
+        final PublisherLease lease = pool.obtainPublisher();
+        assertEquals(0, pool.available());
+
+        lease.poison();
+        lease.close();
+        assertEquals(0, pool.available());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
index 5bc0e0e..e524589 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.kafka;
 
 import java.io.Closeable;
+import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
@@ -111,8 +112,9 @@ class KafkaPublisher implements Closeable {
      *            instance of {@link PublishingContext} which hold context
      *            information about the message(s) to be sent.
      * @return The index of the last successful offset.
+     * @throws IOException if unable to read from the Input Stream
      */
-    KafkaPublisherResult publish(PublishingContext publishingContext) {
+    KafkaPublisherResult publish(PublishingContext publishingContext) throws IOException {
         StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
             publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
index 0a3fe5d..3e01e51 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
@@ -21,17 +21,14 @@ 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 javax.xml.bind.DatatypeConverter;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -39,13 +36,12 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.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.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -53,17 +49,18 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURITY_PROTOCOL;
+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.9 Consumer API. "
+@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.9.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 mean time"
-        + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.")
+        + " 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.")
 @Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.9.x"})
 @WritesAttributes({
     @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"),
     @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY, 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."),
+            + "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")
@@ -75,22 +72,16 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURI
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
 public class ConsumeKafka extends AbstractProcessor {
 
-    private static final long TWO_MB = 2L * 1024L * 1024L;
-
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
 
     static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
 
     static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group");
 
-    static final 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 PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
             .name("topic")
             .displayName("Topic Name(s)")
-            .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma seperated.")
+            .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)
@@ -136,6 +127,7 @@ public class ConsumeKafka extends AbstractProcessor {
                     + "will result in a single FlowFile which  "
                     + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS")
             .build();
+
     static final PropertyDescriptor MAX_POLL_RECORDS = new PropertyDescriptor.Builder()
             .name("max.poll.records")
             .displayName("Max Poll Records")
@@ -145,6 +137,20 @@ public class ConsumeKafka extends AbstractProcessor {
             .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 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.")
@@ -153,8 +159,8 @@ public class ConsumeKafka extends AbstractProcessor {
     static final List<PropertyDescriptor> DESCRIPTORS;
     static final Set<Relationship> RELATIONSHIPS;
 
-    private volatile byte[] demarcatorBytes = null;
     private volatile ConsumerPool consumerPool = null;
+    private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
@@ -165,6 +171,7 @@ public class ConsumeKafka extends AbstractProcessor {
         descriptors.add(KEY_ATTRIBUTE_ENCODING);
         descriptors.add(MESSAGE_DEMARCATOR);
         descriptors.add(MAX_POLL_RECORDS);
+        descriptors.add(MAX_UNCOMMITTED_TIME);
         DESCRIPTORS = Collections.unmodifiableList(descriptors);
         RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
     }
@@ -179,16 +186,8 @@ public class ConsumeKafka extends AbstractProcessor {
         return DESCRIPTORS;
     }
 
-    @OnScheduled
-    public void prepareProcessing(final ProcessContext context) {
-        this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet()
-                ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
-                : null;
-    }
-
     @OnStopped
     public void close() {
-        demarcatorBytes = null;
         final ConsumerPool pool = consumerPool;
         consumerPool = null;
         if (pool != null) {
@@ -215,9 +214,22 @@ public class ConsumeKafka extends AbstractProcessor {
             return pool;
         }
 
-        final Map<String, String> props = new HashMap<>();
+        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.MESSAGE_DEMARCATOR).isSet()
+                ? context.getProperty(ConsumeKafka.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
+                : null;
+
+        final Map<String, Object> props = new HashMap<>();
         KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
-        final String topicListing = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue();
+        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.TOPICS).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
         for (final String topic : topicListing.split(",", 100)) {
             final String trimmedName = topic.trim();
@@ -225,213 +237,78 @@ public class ConsumeKafka extends AbstractProcessor {
                 topics.add(trimmedName);
             }
         }
-        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
-        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        return consumerPool = createConsumerPool(context.getMaxConcurrentTasks(), topics, props, getLogger());
-    }
+        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).getValue();
 
-    protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-        return new ConsumerPool(maxLeases, topics, props, log);
+        return new ConsumerPool(maxLeases, demarcator, props, topics, maxUncommittedTime, keyEncoding, securityProtocol, bootstrapServers, log);
     }
 
-    @Override
-    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-        final long startTimeNanos = System.nanoTime();
-        final ConsumerPool pool = getConsumerPool(context);
-        if (pool == null) {
-            context.yield();
-            return;
-        }
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap = new HashMap<>();
-
-        try (final ConsumerLease lease = pool.obtainConsumer()) {
+    @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 {
-                if (lease == null) {
-                    context.yield();
-                    return;
-                }
-
-                final boolean foundData = gatherDataFromKafka(lease, partitionRecordMap, context);
-                if (!foundData) {
-                    session.rollback();
-                    return;
-                }
-
-                writeSessionData(context, session, partitionRecordMap, startTimeNanos);
-                //At-least once commit handling (if order is reversed it is at-most once)
-                session.commit();
-                commitOffsets(lease, partitionRecordMap);
-            } catch (final KafkaException ke) {
-                lease.poison();
-                getLogger().error("Problem while accessing kafka consumer " + ke, ke);
-                context.yield();
-                session.rollback();
+                Thread.sleep(100L);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                return;
             }
         }
-    }
 
-    private void commitOffsets(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap) {
-        final Map<TopicPartition, OffsetAndMetadata> partOffsetMap = new HashMap<>();
-        partitionRecordMap.entrySet().stream()
-                .filter(entry -> !entry.getValue().isEmpty())
-                .forEach((entry) -> {
-                    long maxOffset = entry.getValue().stream()
-                            .mapToLong(record -> record.offset())
-                            .max()
-                            .getAsLong();
-                    partOffsetMap.put(entry.getKey(), new OffsetAndMetadata(maxOffset + 1L));
-                });
-        lease.commitOffsets(partOffsetMap);
-    }
+        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++;
+            }
 
-    private void writeSessionData(
-            final ProcessContext context, final ProcessSession session,
-            final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap,
-            final long startTimeNanos) {
-        if (demarcatorBytes != null) {
-            partitionRecordMap.entrySet().stream()
-                    .filter(entry -> !entry.getValue().isEmpty())
-                    .forEach(entry -> {
-                        writeData(context, session, entry.getValue(), startTimeNanos);
-                    });
-        } else {
-            partitionRecordMap.entrySet().stream()
-                    .filter(entry -> !entry.getValue().isEmpty())
-                    .flatMap(entry -> entry.getValue().stream())
-                    .forEach(record -> {
-                        writeData(context, session, Collections.singletonList(record), startTimeNanos);
-                    });
+            getLogger().info("Woke up {} consumers", new Object[] {count});
         }
-    }
 
-    private String encodeKafkaKey(final byte[] key, final String encoding) {
-        if (key == null) {
-            return null;
-        }
+        activeLeases.clear();
+    }
 
-        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
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final ConsumerPool pool = getConsumerPool(context);
+        if (pool == null) {
+            context.yield();
+            return;
         }
-    }
 
-    private void writeData(final ProcessContext context, final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final long startTimeNanos) {
-        final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0);
-        final String offset = String.valueOf(firstRecord.offset());
-        final String keyValue = encodeKafkaKey(firstRecord.key(), context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue());
-        final String topic = firstRecord.topic();
-        final String partition = String.valueOf(firstRecord.partition());
-        FlowFile flowFile = session.create();
-        flowFile = session.write(flowFile, out -> {
-            boolean useDemarcator = false;
-            for (final ConsumerRecord<byte[], byte[]> record : records) {
-                if (useDemarcator) {
-                    out.write(demarcatorBytes);
-                }
-                out.write(record.value());
-                useDemarcator = true;
+        try (final ConsumerLease lease = pool.obtainConsumer(session)) {
+            if (lease == null) {
+                context.yield();
+                return;
             }
-        });
-        final Map<String, String> kafkaAttrs = new HashMap<>();
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, offset);
-        if (keyValue != null && records.size() == 1) {
-            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, keyValue);
-        }
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, partition);
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, topic);
-        if (records.size() > 1) {
-            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(records.size()));
-        }
-        flowFile = session.putAllAttributes(flowFile, kafkaAttrs);
-        final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos);
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(
-                context.getProperty(SECURITY_PROTOCOL).getValue(),
-                context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(),
-                topic);
-        session.getProvenanceReporter().receive(flowFile, transitUri, executionDurationMillis);
-        this.getLogger().debug("Created {} containing {} messages from Kafka topic {}, partition {}, starting offset {} in {} millis",
-                new Object[]{flowFile, records.size(), topic, partition, offset, executionDurationMillis});
-        session.transfer(flowFile, REL_SUCCESS);
-    }
 
-    /**
-     * Populates the given partitionRecordMap with new records until we poll
-     * that returns no records or until we have enough data. It is important to
-     * ensure we keep items grouped by their topic and partition so that when we
-     * bundle them we bundle them intelligently and so that we can set offsets
-     * properly even across multiple poll calls.
-     */
-    private boolean gatherDataFromKafka(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, ProcessContext context) {
-        final long startNanos = System.nanoTime();
-        boolean foundData = false;
-        ConsumerRecords<byte[], byte[]> records;
-        final int maxRecords = context.getProperty(MAX_POLL_RECORDS).asInteger();
-
-        do {
-            records = lease.poll();
-
-            for (final TopicPartition partition : records.partitions()) {
-                List<ConsumerRecord<byte[], byte[]>> currList = partitionRecordMap.get(partition);
-                if (currList == null) {
-                    currList = new ArrayList<>();
-                    partitionRecordMap.put(partition, currList);
+            activeLeases.add(lease);
+            try {
+                while (this.isScheduled() && lease.continuePolling()) {
+                    lease.poll();
                 }
-                currList.addAll(records.records(partition));
-                if (currList.size() > 0) {
-                    foundData = true;
+                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);
             }
-            //If we received data and we still want to get more
-        } while (!records.isEmpty() && !checkIfGatheredEnoughData(partitionRecordMap, maxRecords, startNanos));
-        return foundData;
-    }
-
-    /**
-     * Determines if we have enough data as-is and should move on.
-     *
-     * @return true if we've been gathering for more than 500 ms or if we're
-     * demarcating and have more than 50 flowfiles worth or if we're per message
-     * and have more than 2000 flowfiles or if totalMessageSize is greater than
-     * two megabytes; false otherwise
-     *
-     * Implementation note: 500 millis and 5 MB are magic numbers. These may
-     * need to be tuned. They get at how often offsets will get committed to
-     * kafka relative to how many records will get buffered into memory in a
-     * poll call before writing to repos.
-     */
-    private boolean checkIfGatheredEnoughData(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, final long maxRecords, final long startTimeNanos) {
-
-        final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos);
-
-        if (durationMillis > 500) {
-            return true;
-        }
-
-        int topicPartitionsFilled = 0;
-        int totalRecords = 0;
-        long totalRecordSize = 0;
-
-        for (final List<ConsumerRecord<byte[], byte[]>> recordList : partitionRecordMap.values()) {
-            if (!recordList.isEmpty()) {
-                topicPartitionsFilled++;
-            }
-            totalRecords += recordList.size();
-            for (final ConsumerRecord<byte[], byte[]> rec : recordList) {
-                totalRecordSize += rec.value().length;
-            }
-        }
-
-        if (demarcatorBytes != null && demarcatorBytes.length > 0) {
-            return topicPartitionsFilled > 50;
-        } else if (totalRecordSize > TWO_MB) {
-            return true;
-        } else {
-            return totalRecords > maxRecords;
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
index b954eba..cd9365d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
@@ -17,11 +17,28 @@
 package org.apache.nifi.processors.kafka.pubsub;
 
 import java.io.Closeable;
+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.concurrent.TimeUnit;
+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.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafka.REL_SUCCESS;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 /**
  * This class represents a lease to access a Kafka Consumer object. The lease is
@@ -30,15 +47,108 @@ import org.apache.kafka.common.TopicPartition;
  * the lease will be returned to the pool for future use by others. A given
  * lease may only belong to a single thread a time.
  */
-public interface ConsumerLease extends Closeable {
+public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListener {
+
+    private final long maxWaitMillis;
+    private final Consumer<byte[], byte[]> kafkaConsumer;
+    private final ComponentLog logger;
+    private final byte[] demarcatorBytes;
+    private final String keyEncoding;
+    private final String securityProtocol;
+    private final String bootstrapServers;
+    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<TopicPartition, BundleTracker> bundleMap = new HashMap<>();
+    private final Map<TopicPartition, OffsetAndMetadata> uncommittedOffsetsMap = new HashMap<>();
+    private long leaseStartNanos = -1;
+    private boolean lastPollEmpty = false;
+    private int totalFlowFiles = 0;
+
+    ConsumerLease(
+            final long maxWaitMillis,
+            final Consumer<byte[], byte[]> kafkaConsumer,
+            final byte[] demarcatorBytes,
+            final String keyEncoding,
+            final String securityProtocol,
+            final String bootstrapServers,
+            final ComponentLog logger) {
+        this.maxWaitMillis = maxWaitMillis;
+        this.kafkaConsumer = kafkaConsumer;
+        this.demarcatorBytes = demarcatorBytes;
+        this.keyEncoding = keyEncoding;
+        this.securityProtocol = securityProtocol;
+        this.bootstrapServers = bootstrapServers;
+        this.logger = logger;
+    }
+
+    /**
+     * 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;
+        totalFlowFiles = 0;
+    }
 
     /**
-     * Executes a poll on the underlying Kafka Consumer.
+     * 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.
      *
-     * @return ConsumerRecords retrieved in the poll.
-     * @throws KafkaException if issue occurs talking to underlying resource.
+     * @param partitions partitions being reassigned
+     */
+    @Override
+    public void onPartitionsRevoked(final Collection<TopicPartition> 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 parittion
+        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<TopicPartition> 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.
      */
-    ConsumerRecords<byte[], byte[]> poll() throws KafkaException;
+    void poll() {
+        /**
+         * Implementation note: If we take too long (30 secs?) between kafka
+         * poll calls and our own record processing to any subsequent poll calls
+         * or the commit we can run into a situation where the commit will
+         * succeed to the session but fail on committing offsets. This is
+         * apparently different than the Kafka scenario of electing to rebalance
+         * for other reasons but in this case is due a session timeout. It
+         * appears Kafka KIP-62 aims to offer more control over the meaning of
+         * various timeouts. If we do run into this case it could result in
+         * duplicates.
+         */
+        try {
+            final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(10);
+            lastPollEmpty = records.count() == 0;
+            processRecords(records);
+        } catch (final Throwable t) {
+            this.poison();
+            throw t;
+        }
+    }
 
     /**
      * Notifies Kafka to commit the offsets for the specified topic/partition
@@ -47,22 +157,251 @@ public interface ConsumerLease extends Closeable {
      * kafka client to collect more data from Kafka before committing the
      * offsets.
      *
-     * @param offsets offsets
-     * @throws KafkaException if issue occurs talking to underlying resource.
+     * if false then we didn't do anything and should probably yield if true
+     * then we committed new data
+     *
      */
-    void commitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) throws KafkaException;
+    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<FlowFile> bundledFlowFiles = getBundles();
+            if (!bundledFlowFiles.isEmpty()) {
+                getProcessSession().transfer(bundledFlowFiles, REL_SUCCESS);
+            }
+            getProcessSession().commit();
+            kafkaConsumer.commitSync(uncommittedOffsetsMap);
+            resetInternalState();
+            return true;
+        } 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;
+        }
+    }
 
     /**
-     * Notifies that this lease is poisoned and should not be reused.
+     * 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
      */
-    void poison();
+    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 totalFlowFiles < 15000;//admittedly a magic number - good candidate for processor property
+        }
+    }
 
     /**
-     * Notifies that this lease is to be returned. The pool may optionally reuse
-     * this lease with another client. No further references by the caller
-     * should occur after calling close.
+     * 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
-    void close();
+    public void close() {
+        resetInternalState();
+    }
+
+    public abstract ProcessSession getProcessSession();
+
+    private void processRecords(final ConsumerRecords<byte[], byte[]> records) {
+
+        records.partitions().stream().forEach(partition -> {
+            List<ConsumerRecord<byte[], byte[]>> 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();
+                uncommittedOffsetsMap.put(partition, new OffsetAndMetadata(maxOffset + 1L));
+
+                //write records to content repository and session
+                if (demarcatorBytes == null) {
+                    totalFlowFiles += messages.size();
+                    messages.stream().forEach(message -> {
+                        writeData(getProcessSession(), message, partition);
+                    });
+                } else {
+                    writeData(getProcessSession(), messages, partition);
+                }
+            }
+        });
+    }
+
+    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<FlowFile> getBundles() {
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        for (final BundleTracker tracker : bundleMap.values()) {
+            populateAttributes(tracker);
+            flowFiles.add(tracker.flowFile);
+        }
+        return flowFiles;
+    }
+
+    private void writeData(final ProcessSession session, ConsumerRecord<byte[], byte[]> record, final TopicPartition topicPartition) {
+        FlowFile flowFile = session.create();
+        final BundleTracker tracker = new BundleTracker(record, topicPartition, keyEncoding);
+        tracker.incrementRecordCount(1);
+        flowFile = session.write(flowFile, out -> {
+            out.write(record.value());
+        });
+        tracker.updateFlowFile(flowFile);
+        populateAttributes(tracker);
+        session.transfer(tracker.flowFile, REL_SUCCESS);
+    }
+
+    private void writeData(final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final TopicPartition topicPartition) {
+        final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0);
+        final boolean demarcateFirstRecord;
+        BundleTracker tracker = bundleMap.get(topicPartition);
+        FlowFile flowFile;
+        if (tracker == null) {
+            tracker = new BundleTracker(firstRecord, topicPartition, keyEncoding);
+            flowFile = session.create();
+            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(records.size());
+        flowFile = session.append(flowFile, out -> {
+            boolean useDemarcator = demarcateFirstRecord;
+            for (final ConsumerRecord<byte[], byte[]> record : records) {
+                if (useDemarcator) {
+                    out.write(demarcatorBytes);
+                }
+                out.write(record.value());
+                useDemarcator = true;
+            }
+        });
+        tracker.updateFlowFile(flowFile);
+        bundleMap.put(topicPartition, tracker);
+    }
+
+    private void populateAttributes(final BundleTracker tracker) {
+        final Map<String, String> 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) {
+            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_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;
+        FlowFile flowFile;
+        long totalRecords = 0;
+
+        private BundleTracker(final ConsumerRecord<byte[], byte[]> initialRecord, final TopicPartition topicPartition, final String keyEncoding) {
+            this.initialOffset = initialRecord.offset();
+            this.partition = topicPartition.partition();
+            this.topic = topicPartition.topic();
+            this.key = encodeKafkaKey(initialRecord.key(), keyEncoding);
+        }
+
+        private void incrementRecordCount(final long count) {
+            totalRecords += count;
+        }
+
+        private void updateFlowFile(final FlowFile flowFile) {
+            this.flowFile = flowFile;
+        }
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
index 3f20b8f..e13a8c3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
@@ -21,18 +21,15 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.nifi.logging.ComponentLog;
 
 import java.io.Closeable;
-import java.util.ArrayDeque;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.atomic.AtomicLong;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
+import org.apache.nifi.processor.ProcessSession;
 
 /**
  * A pool of Kafka Consumers for a given topic. Consumers can be obtained by
@@ -41,176 +38,119 @@ import org.apache.kafka.common.TopicPartition;
  */
 public class ConsumerPool implements Closeable {
 
-    private final AtomicInteger activeLeaseCount = new AtomicInteger(0);
-    private final int maxLeases;
-    private final Queue<ConsumerLease> consumerLeases;
+    private final BlockingQueue<SimpleConsumerLease> pooledLeases;
     private final List<String> topics;
     private final Map<String, Object> 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 AtomicLong consumerCreatedCountRef = new AtomicLong();
     private final AtomicLong consumerClosedCountRef = new AtomicLong();
     private final AtomicLong leasesObtainedCountRef = new AtomicLong();
-    private final AtomicLong productivePollCountRef = new AtomicLong();
-    private final AtomicLong unproductivePollCountRef = new AtomicLong();
 
     /**
      * Creates a pool of KafkaConsumer objects that will grow up to the maximum
-     * indicated leases. Consumers are lazily initialized.
+     * 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 maxLeases maximum number of active leases in the pool
-     * @param topics the topics to consume from
-     * @param kafkaProperties the properties for each consumer
+     * @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 maxLeases, final List<String> topics, final Map<String, String> kafkaProperties, final ComponentLog logger) {
-        this.maxLeases = maxLeases;
-        if (maxLeases <= 0) {
-            throw new IllegalArgumentException("Max leases value must be greather than zero.");
-        }
+    public ConsumerPool(
+            final int maxConcurrentLeases,
+            final byte[] demarcator,
+        final Map<String, Object> kafkaProperties,
+            final List<String> topics,
+            final long maxWaitMillis,
+            final String keyEncoding,
+            final String securityProtocol,
+            final String bootstrapServers,
+            final ComponentLog logger) {
+        this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases);
+        this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
-        if (topics == null || topics.isEmpty()) {
-            throw new IllegalArgumentException("Must have a list of one or more topics");
-        }
-        this.topics = topics;
-        this.kafkaProperties = new HashMap<>(kafkaProperties);
-        this.consumerLeases = new ArrayDeque<>();
+        this.demarcatorBytes = demarcator;
+        this.keyEncoding = keyEncoding;
+        this.securityProtocol = securityProtocol;
+        this.bootstrapServers = bootstrapServers;
+        this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties);
+        this.topics = Collections.unmodifiableList(topics);
     }
 
     /**
-     * Obtains a consumer from the pool if one is available
+     * Obtains a consumer from the pool if one is available or lazily
+     * initializes a new one if deemed necessary.
      *
-     * @return consumer from the pool
-     * @throws IllegalArgumentException if pool already contains
+     * @param session the session for which the consumer lease will be
+     * associated
+     * @return consumer to use or null if not available or necessary
      */
-    public ConsumerLease obtainConsumer() {
-        final ConsumerLease lease;
-        final int activeLeases;
-        synchronized (this) {
-            lease = consumerLeases.poll();
-            activeLeases = activeLeaseCount.get();
-        }
-        if (lease == null && activeLeases >= maxLeases) {
-            logger.warn("No available consumers and cannot create any as max consumer leases limit reached - verify pool settings");
-            return null;
+    public ConsumerLease obtainConsumer(final ProcessSession session) {
+        SimpleConsumerLease lease = pooledLeases.poll();
+        if (lease == null) {
+            final Consumer<byte[], byte[]> 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.
+             */
+            consumer.subscribe(topics, lease);
         }
+        lease.setProcessSession(session);
         leasesObtainedCountRef.incrementAndGet();
-        return (lease == null) ? createConsumer() : lease;
+        return lease;
     }
 
+    /**
+     * Exposed as protected method for easier unit testing
+     *
+     * @return consumer
+     * @throws KafkaException if unable to subscribe to the given topics
+     */
     protected Consumer<byte[], byte[]> createKafkaConsumer() {
         return new KafkaConsumer<>(kafkaProperties);
     }
 
-    private ConsumerLease createConsumer() {
-        final Consumer<byte[], byte[]> kafkaConsumer = createKafkaConsumer();
-        consumerCreatedCountRef.incrementAndGet();
-        try {
-            kafkaConsumer.subscribe(topics);
-        } catch (final KafkaException kex) {
-            try {
-                kafkaConsumer.close();
-                consumerClosedCountRef.incrementAndGet();
-            } catch (final Exception ex) {
-                consumerClosedCountRef.incrementAndGet();
-                //ignore
-            }
-            throw kex;
-        }
-
-        final ConsumerLease lease = new ConsumerLease() {
-
-            private volatile boolean poisoned = false;
-            private volatile boolean closed = false;
-
-            @Override
-            public ConsumerRecords<byte[], byte[]> poll() {
-
-                if (poisoned) {
-                    throw new KafkaException("The consumer is poisoned and should no longer be used");
-                }
-
-                try {
-                    final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(50);
-                    if (records.isEmpty()) {
-                        unproductivePollCountRef.incrementAndGet();
-                    } else {
-                        productivePollCountRef.incrementAndGet();
-                    }
-                    return records;
-                } catch (final KafkaException kex) {
-                    logger.warn("Unable to poll from Kafka consumer so will poison and close this " + kafkaConsumer, kex);
-                    poison();
-                    close();
-                    throw kex;
-                }
-            }
-
-            @Override
-            public void commitOffsets(final Map<TopicPartition, OffsetAndMetadata> offsets) {
-
-                if (poisoned) {
-                    throw new KafkaException("The consumer is poisoned and should no longer be used");
-                }
-                try {
-                    kafkaConsumer.commitSync(offsets);
-                } catch (final KafkaException kex) {
-                    logger.warn("Unable to commit kafka consumer offsets so will poison and close this " + kafkaConsumer, kex);
-                    poison();
-                    close();
-                    throw kex;
-                }
-            }
-
-            @Override
-            public void close() {
-                if (closed) {
-                    return;
-                }
-                if (poisoned || activeLeaseCount.get() > maxLeases) {
-                    closeConsumer(kafkaConsumer);
-                    activeLeaseCount.decrementAndGet();
-                    closed = true;
-                } else {
-                    final boolean added;
-                    synchronized (ConsumerPool.this) {
-                        added = consumerLeases.offer(this);
-                    }
-                    if (!added) {
-                        closeConsumer(kafkaConsumer);
-                        activeLeaseCount.decrementAndGet();
-                    }
-                }
-            }
-
-            @Override
-            public void poison() {
-                poisoned = true;
-            }
-        };
-        activeLeaseCount.incrementAndGet();
-        return lease;
-    }
-
     /**
-     * Closes all consumers in the pool. Can be safely recalled.
+     * Closes all consumers in the pool. Can be safely called repeatedly.
      */
     @Override
     public void close() {
-        final List<ConsumerLease> leases = new ArrayList<>();
-        synchronized (this) {
-            ConsumerLease lease = null;
-            while ((lease = consumerLeases.poll()) != null) {
-                leases.add(lease);
-            }
-        }
-        for (final ConsumerLease lease : leases) {
-            lease.poison();
-            lease.close();
-        }
+        final List<SimpleConsumerLease> leases = new ArrayList<>();
+        pooledLeases.drainTo(leases);
+        leases.stream().forEach((lease) -> {
+            lease.close(true);
+        });
     }
 
-    private void closeConsumer(final Consumer consumer) {
+    private void closeConsumer(final Consumer<?, ?> consumer) {
+        consumerClosedCountRef.incrementAndGet();
         try {
             consumer.unsubscribe();
         } catch (Exception e) {
@@ -219,15 +159,55 @@ public class ConsumerPool implements Closeable {
 
         try {
             consumer.close();
-            consumerClosedCountRef.incrementAndGet();
         } catch (Exception e) {
-            consumerClosedCountRef.incrementAndGet();
             logger.warn("Failed while closing " + consumer, e);
         }
     }
 
     PoolStats getPoolStats() {
-        return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get(), productivePollCountRef.get(), unproductivePollCountRef.get());
+        return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get());
+    }
+
+    private class SimpleConsumerLease extends ConsumerLease {
+
+        private final Consumer<byte[], byte[]> consumer;
+        private volatile ProcessSession session;
+        private volatile boolean closedConsumer;
+
+        private SimpleConsumerLease(final Consumer<byte[], byte[]> consumer) {
+            super(maxWaitMillis, consumer, demarcatorBytes, keyEncoding, securityProtocol, bootstrapServers, logger);
+            this.consumer = consumer;
+        }
+
+        void setProcessSession(final ProcessSession session) {
+            this.session = session;
+        }
+
+        @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);
+            }
+            if (forceClose || isPoisoned() || !pooledLeases.offer(this)) {
+                closedConsumer = true;
+                closeConsumer(consumer);
+            }
+        }
     }
 
     static final class PoolStats {
@@ -235,30 +215,22 @@ public class ConsumerPool implements Closeable {
         final long consumerCreatedCount;
         final long consumerClosedCount;
         final long leasesObtainedCount;
-        final long productivePollCount;
-        final long unproductivePollCount;
 
         PoolStats(
                 final long consumerCreatedCount,
                 final long consumerClosedCount,
-                final long leasesObtainedCount,
-                final long productivePollCount,
-                final long unproductivePollCount
+                final long leasesObtainedCount
         ) {
             this.consumerCreatedCount = consumerCreatedCount;
             this.consumerClosedCount = consumerClosedCount;
             this.leasesObtainedCount = leasesObtainedCount;
-            this.productivePollCount = productivePollCount;
-            this.unproductivePollCount = unproductivePollCount;
         }
 
         @Override
         public String toString() {
             return "Created Consumers [" + consumerCreatedCount + "]\n"
                     + "Closed Consumers  [" + consumerClosedCount + "]\n"
-                    + "Leases Obtained   [" + leasesObtainedCount + "]\n"
-                    + "Productive Polls  [" + productivePollCount + "]\n"
-                    + "Unproductive Polls  [" + unproductivePollCount + "]\n";
+                    + "Leases Obtained   [" + leasesObtainedCount + "]\n";
         }
 
     }


[03/20] nifi git commit: NIFI-2704: - Re-using the DataTransferAuthorizable in the DataTransferResource. - Removing use of the DataTransferResource when obtaining site to site details as it performs additional unnecessary checks. - Code clean up. - This

Posted by jp...@apache.org.
NIFI-2704:
- Re-using the DataTransferAuthorizable in the DataTransferResource.
- Removing use of the DataTransferResource when obtaining site to site details as it performs additional unnecessary checks.
- Code clean up.
- This closes #971.


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

Branch: refs/heads/support/nifi-1.0.x
Commit: bde1c240faa64b83b68887c9455302ea89374c9f
Parents: 94bc7c6
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Aug 30 14:10:41 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:20:34 2016 -0500

----------------------------------------------------------------------
 .../nifi/web/StandardNiFiServiceFacade.java     |  29 +++-
 .../nifi/web/api/DataTransferResource.java      | 141 ++++++++++---------
 .../apache/nifi/web/api/SiteToSiteResource.java |  23 ++-
 .../src/main/resources/nifi-web-api-context.xml |   1 +
 .../nifi/web/api/TestDataTransferResource.java  |   7 +-
 5 files changed, 119 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/bde1c240/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index dfa3e95..e7196e4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -26,6 +26,7 @@ import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.AccessPolicy;
 import org.apache.nifi.authorization.AuthorizableLookup;
+import org.apache.nifi.authorization.AuthorizationRequest;
 import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.AuthorizeAccess;
@@ -34,8 +35,8 @@ import org.apache.nifi.authorization.Group;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserContextKeys;
 import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -2449,7 +2450,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     /**
-     * Ensures the specified user has permission to access the specified port.
+     * Ensures the specified user has permission to access the specified port. This method does
+     * not utilize the DataTransferAuthorizable as that will enforce the entire chain is
+     * authorized for the transfer. This method is only invoked when obtaining the site to site
+     * details so the entire chain isn't necessary.
      */
     private boolean isUserAuthorized(final NiFiUser user, final RootGroupPort port) {
         final boolean isSiteToSiteSecure = Boolean.TRUE.equals(properties.isSiteToSiteSecure());
@@ -2459,9 +2463,24 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             return true;
         }
 
-        // authorize this port for data transfer
-        final Authorizable dataTransferAuthorizable = new DataTransferAuthorizable(port);
-        final AuthorizationResult result = dataTransferAuthorizable.checkAuthorization(authorizer, RequestAction.WRITE, user);
+        final Map<String, String> userContext;
+        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
+            userContext = new HashMap<>();
+            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
+        } else {
+            userContext = null;
+        }
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder()
+                .resource(ResourceFactory.getDataTransferResource(port.getResource()))
+                .identity(user.getIdentity())
+                .anonymous(user.isAnonymous())
+                .accessAttempt(false)
+                .action(RequestAction.WRITE)
+                .userContext(userContext)
+                .build();
+
+        final AuthorizationResult result = authorizer.authorize(request);
         return Result.Approved.equals(result.getResult());
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/bde1c240/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
index 7e0aff9..f859b8e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/DataTransferResource.java
@@ -16,57 +16,17 @@
  */
 package org.apache.nifi.web.api;
 
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_COUNT;
-import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_DURATION;
-import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_SIZE;
-import static org.apache.nifi.remote.protocol.HandshakeProperty.REQUEST_EXPIRATION_MILLIS;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_COUNT;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_DURATION;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
-
 import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiParam;
 import com.wordnik.swagger.annotations.ApiResponse;
 import com.wordnik.swagger.annotations.ApiResponses;
 import com.wordnik.swagger.annotations.Authorization;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.StreamingOutput;
-import javax.ws.rs.core.UriInfo;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.authorization.AccessDeniedException;
-import org.apache.nifi.authorization.AuthorizationRequest;
-import org.apache.nifi.authorization.AuthorizationResult;
-import org.apache.nifi.authorization.AuthorizationResult.Result;
+import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
-import org.apache.nifi.authorization.Resource;
-import org.apache.nifi.authorization.UserContextKeys;
-import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
 import org.apache.nifi.authorization.resource.ResourceType;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -89,10 +49,45 @@ import org.apache.nifi.remote.protocol.http.HttpFlowFileServerProtocol;
 import org.apache.nifi.remote.protocol.http.StandardHttpFlowFileServerProtocol;
 import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.entity.TransactionResultEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.StreamingOutput;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_COUNT;
+import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_DURATION;
+import static org.apache.nifi.remote.protocol.HandshakeProperty.BATCH_SIZE;
+import static org.apache.nifi.remote.protocol.HandshakeProperty.REQUEST_EXPIRATION_MILLIS;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_COUNT;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_DURATION;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
+
 /**
  * RESTful endpoint for managing a SiteToSite connection.
  */
@@ -113,6 +108,7 @@ public class DataTransferResource extends ApplicationResource {
     private static final String PORT_TYPE_OUTPUT = "output-ports";
 
     private Authorizer authorizer;
+    private NiFiServiceFacade serviceFacade;
     private final ResponseCreator responseCreator = new ResponseCreator();
     private final VersionNegotiator transportProtocolVersionNegotiator = new TransportProtocolVersionNegotiator(1);
     private final HttpRemoteSiteListener transactionManager;
@@ -128,36 +124,24 @@ public class DataTransferResource extends ApplicationResource {
      * <p>
      * Note: Protected for testing purposes
      */
-    protected void authorizeDataTransfer(final ResourceType resourceType, final String identifier) {
+    protected void authorizeDataTransfer(final AuthorizableLookup lookup, final ResourceType resourceType, final String identifier) {
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
+        // ensure the resource type is correct
         if (!ResourceType.InputPort.equals(resourceType) && !ResourceType.OutputPort.equals(resourceType)) {
             throw new IllegalArgumentException("The resource must be an Input or Output Port.");
         }
 
-        final Map<String, String> userContext;
-        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
-            userContext = new HashMap<>();
-            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
+        // get the authorizable
+        final DataTransferAuthorizable authorizable;
+        if (ResourceType.InputPort.equals(resourceType)) {
+            authorizable = new DataTransferAuthorizable(lookup.getInputPort(identifier));
         } else {
-            userContext = null;
+            authorizable = new DataTransferAuthorizable(lookup.getOutputPort(identifier));
         }
 
-        final Resource resource = ResourceFactory.getComponentResource(resourceType, identifier, identifier);
-        final AuthorizationRequest request = new AuthorizationRequest.Builder()
-                .resource(ResourceFactory.getDataTransferResource(resource))
-                .identity(user.getIdentity())
-                .anonymous(user.isAnonymous())
-                .accessAttempt(true)
-                .action(RequestAction.WRITE)
-                .userContext(userContext)
-                .build();
-
-        final AuthorizationResult result = authorizer.authorize(request);
-        if (!Result.Approved.equals(result.getResult())) {
-            final String message = StringUtils.isNotBlank(result.getExplanation()) ? result.getExplanation() : "Access is denied";
-            throw new AccessDeniedException(message);
-        }
+        // perform the authorization
+        authorizable.authorize(authorizer, RequestAction.WRITE, user);
     }
 
     @POST
@@ -199,7 +183,9 @@ public class DataTransferResource extends ApplicationResource {
         }
 
         // authorize access
-        authorizeDataTransfer(PORT_TYPE_INPUT.equals(portType) ? ResourceType.InputPort : ResourceType.OutputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, PORT_TYPE_INPUT.equals(portType) ? ResourceType.InputPort : ResourceType.OutputPort, portId);
+        });
 
         final ValidateRequestResult validationResult = validateResult(req, portId);
         if (validationResult.errResponse != null) {
@@ -266,7 +252,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.InputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.InputPort, portId);
+        });
 
         final ValidateRequestResult validationResult = validateResult(req, portId, transactionId);
         if (validationResult.errResponse != null) {
@@ -421,7 +409,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.OutputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.OutputPort, portId);
+        });
 
         final ValidateRequestResult validationResult = validateResult(req, portId, transactionId);
         if (validationResult.errResponse != null) {
@@ -523,7 +513,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.InputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.InputPort, portId);
+        });
 
         final ValidateRequestResult validationResult = validateResult(req, portId, transactionId);
         if (validationResult.errResponse != null) {
@@ -634,7 +626,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.OutputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.OutputPort, portId);
+        });
 
         final ValidateRequestResult validationResult = validateResult(req, portId, transactionId);
         if (validationResult.errResponse != null) {
@@ -713,7 +707,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.InputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.InputPort, portId);
+        });
 
         return extendPortTransactionTTL(PORT_TYPE_INPUT, portId, transactionId, req, res, context, uriInfo, inputStream);
     }
@@ -749,7 +745,9 @@ public class DataTransferResource extends ApplicationResource {
             InputStream inputStream) {
 
         // authorize access
-        authorizeDataTransfer(ResourceType.OutputPort, portId);
+        serviceFacade.authorizeAccess(lookup -> {
+            authorizeDataTransfer(lookup, ResourceType.OutputPort, portId);
+        });
 
         return extendPortTransactionTTL(PORT_TYPE_OUTPUT, portId, transactionId, req, res, context, uriInfo, inputStream);
     }
@@ -837,4 +835,7 @@ public class DataTransferResource extends ApplicationResource {
         this.authorizer = authorizer;
     }
 
+    public void setServiceFacade(NiFiServiceFacade serviceFacade) {
+        this.serviceFacade = serviceFacade;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/bde1c240/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
index 88bdeb6..63e5a35 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SiteToSiteResource.java
@@ -17,9 +17,6 @@
 package org.apache.nifi.web.api;
 
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
 import com.wordnik.swagger.annotations.Api;
 import com.wordnik.swagger.annotations.ApiOperation;
 import com.wordnik.swagger.annotations.ApiResponse;
@@ -32,6 +29,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.UserContextKeys;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -43,6 +41,7 @@ import org.apache.nifi.remote.VersionNegotiator;
 import org.apache.nifi.remote.client.http.TransportProtocolVersionNegotiator;
 import org.apache.nifi.remote.exception.BadRequestException;
 import org.apache.nifi.remote.protocol.http.HttpHeaders;
+import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.dto.ControllerDTO;
 import org.apache.nifi.web.api.dto.remote.PeerDTO;
@@ -60,12 +59,15 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import static org.apache.commons.lang3.StringUtils.isEmpty;
-import org.apache.nifi.util.NiFiProperties;
 
 /**
  * RESTful endpoint for managing a SiteToSite connection.
@@ -99,12 +101,21 @@ public class SiteToSiteResource extends ApplicationResource {
     protected void authorizeSiteToSite() {
         final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
+        final Map<String, String> userContext;
+        if (!StringUtils.isBlank(user.getClientAddress())) {
+            userContext = new HashMap<>();
+            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
+        } else {
+            userContext = null;
+        }
+
         final AuthorizationRequest request = new AuthorizationRequest.Builder()
                 .resource(ResourceFactory.getSiteToSiteResource())
                 .identity(user.getIdentity())
                 .anonymous(user.isAnonymous())
                 .accessAttempt(true)
                 .action(RequestAction.READ)
+                .userContext(userContext)
                 .build();
 
         final AuthorizationResult result = authorizer.authorize(request);
@@ -155,7 +166,7 @@ public class SiteToSiteResource extends ApplicationResource {
         if (isEmpty(req.getHeader(HttpHeaders.PROTOCOL_VERSION))) {
             // This indicates the client uses older NiFi version,
             // which strictly read JSON properties and fail with unknown properties.
-            // Convert result entity so that old version clients can understance.
+            // Convert result entity so that old version clients can understand.
             logger.debug("Converting result to provide backward compatibility...");
             controller.setRemoteSiteHttpListeningPort(null);
         }
@@ -173,7 +184,6 @@ public class SiteToSiteResource extends ApplicationResource {
     @Path("/peers")
     @Consumes(MediaType.WILDCARD)
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    // TODO: @PreAuthorize("hasRole('ROLE_NIFI')")
     @ApiOperation(
             value = "Returns the available Peers and its status of this NiFi",
             response = PeersEntity.class,
@@ -251,6 +261,7 @@ public class SiteToSiteResource extends ApplicationResource {
     }
 
     // setters
+
     public void setServiceFacade(final NiFiServiceFacade serviceFacade) {
         this.serviceFacade = serviceFacade;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/bde1c240/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 104067c..2f3fb29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -228,6 +228,7 @@
         <property name="clusterCoordinator" ref="clusterCoordinator"/>
         <property name="requestReplicator" ref="requestReplicator" />
         <property name="authorizer" ref="authorizer"/>
+        <property name="serviceFacade" ref="serviceFacade"/>
     </bean>
     <bean id="snippetResource" class="org.apache.nifi.web.api.SnippetResource" scope="singleton">
         <property name="serviceFacade" ref="serviceFacade"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/bde1c240/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
index eb87830..3dac9ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.web.api;
 
+import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.resource.ResourceType;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
 import org.apache.nifi.remote.Peer;
@@ -27,6 +28,7 @@ import org.apache.nifi.remote.protocol.ResponseCode;
 import org.apache.nifi.remote.protocol.http.HttpFlowFileServerProtocol;
 import org.apache.nifi.remote.protocol.http.HttpHeaders;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.entity.TransactionResultEntity;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -341,10 +343,12 @@ public class TestDataTransferResource {
     }
 
     private DataTransferResource getDataTransferResource() {
+        final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
+
         final HttpFlowFileServerProtocol serverProtocol = mock(HttpFlowFileServerProtocol.class);
         final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties(null, null)) {
             @Override
-            protected void authorizeDataTransfer(ResourceType resourceType, String identifier) {
+            protected void authorizeDataTransfer(AuthorizableLookup lookup, ResourceType resourceType, String identifier) {
             }
 
             @Override
@@ -353,6 +357,7 @@ public class TestDataTransferResource {
             }
         };
         resource.setProperties(NiFiProperties.createBasicNiFiProperties(null, null));
+        resource.setServiceFacade(serviceFacade);
         return resource;
     }
 }
\ No newline at end of file


[18/20] nifi git commit: NIFI-2768: - Ensuring we correctly enforce authorization when the destination of a connection changes. This includes source, destination, and parent process group. - Ensuring the UI reseting accordingly when a request fails.

Posted by jp...@apache.org.
NIFI-2768: - Ensuring we correctly enforce authorization when the destination of a connection changes. This includes source, destination, and parent process group. - Ensuring the UI reseting accordingly when a request fails.

This closes #1014.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/support/nifi-1.0.x
Commit: cdfed075a1563695a15d088a3ce5a10a3f73f99f
Parents: af1732a
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Sep 13 11:22:03 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Thu Dec 15 10:21:03 2016 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/web/api/ConnectionResource.java  | 2 +-
 .../nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js      | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/cdfed075/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
index 11abf86..fb25211 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java
@@ -223,7 +223,7 @@ public class ConnectionResource extends ApplicationResource {
 
                     // if a destination has been specified and is different
                     final Connectable currentDestination = connAuth.getDestination();
-                    if (requestConnection.getDestination() != null && currentDestination.getIdentifier().equals(requestConnection.getDestination().getId())) {
+                    if (requestConnection.getDestination() != null && !currentDestination.getIdentifier().equals(requestConnection.getDestination().getId())) {
                         // verify access of the new destination (current destination was already authorized as part of the connection check)
                         final Authorizable newDestinationAuthorizable = lookup.getConnectable(requestConnection.getDestination().getId());
                         newDestinationAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

http://git-wip-us.apache.org/repos/asf/nifi/blob/cdfed075/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js
index 01d411a..dcc54f1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection.js
@@ -1386,7 +1386,7 @@ nf.Connection = (function () {
                                 nf.CanvasUtils.reloadConnectionSourceAndDestination(null, previousDestinationId);
                                 nf.CanvasUtils.reloadConnectionSourceAndDestination(response.sourceId, response.destinationId);
                             }).fail(function (xhr, status, error) {
-                                if (xhr.status === 400 || xhr.status === 404 || xhr.status === 409) {
+                                if (xhr.status === 400 || xhr.status === 401 || xhr.status === 403 || xhr.status === 404 || xhr.status === 409) {
                                     nf.Dialog.showOkDialog({
                                         headerText: 'Connection',
                                         dialogContent: nf.Common.escapeHtml(xhr.responseText)


[06/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
new file mode 100644
index 0000000..f9f2485
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java
@@ -0,0 +1,262 @@
+/*
+ * 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.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.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() {
+            @Override
+            protected PublisherPool createPublisherPool(final ProcessContext context) {
+                return mockPool;
+            }
+        });
+
+        runner.setProperty(PublishKafka.TOPIC, TOPIC_NAME);
+    }
+
+    @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.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<FlowFile> 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.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.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<FlowFile> 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.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<FlowFile> flowFiles = new ArrayList<>();
+        flowFiles.add(runner.enqueue("hello world"));
+        flowFiles.add(runner.enqueue("hello world"));
+
+        final Map<FlowFile, Integer> 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.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.REL_SUCCESS).stream()
+            .filter(ff -> ff.getAttribute("msg.count").equals("10"))
+            .count());
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
+            .filter(ff -> ff.getAttribute("msg.count").equals("20"))
+            .count());
+    }
+
+
+    @Test
+    public void testSomeSuccessSomeFailure() throws IOException {
+        final List<FlowFile> 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<FlowFile, Integer> msgCounts = new HashMap<>();
+        msgCounts.put(flowFiles.get(0), 10);
+        msgCounts.put(flowFiles.get(1), 20);
+
+        final Map<FlowFile, Exception> 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.REL_SUCCESS, 2);
+        runner.assertTransferCount(PublishKafka.REL_FAILURE, 2);
+
+        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();
+
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
+            .filter(ff -> "10".equals(ff.getAttribute("msg.count")))
+            .count());
+        assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
+            .filter(ff -> "20".equals(ff.getAttribute("msg.count")))
+            .count());
+
+        assertTrue(runner.getFlowFilesForRelationship(PublishKafka.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<FlowFile> successfulFlowFiles, final int msgCountPerFlowFile) {
+        final Map<FlowFile, Integer> 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<FlowFile> failures) {
+        final Map<FlowFile, Exception> 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<FlowFile, Integer> msgCounts, final Set<FlowFile> successFlowFiles, final Map<FlowFile, Exception> 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 Collection<FlowFile> getSuccessfulFlowFiles() {
+                return successFlowFiles;
+            }
+
+            @Override
+            public Collection<FlowFile> getFailedFlowFiles() {
+                return failures.keySet();
+            }
+
+            @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);
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
new file mode 100644
index 0000000..c2d143c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java
@@ -0,0 +1,194 @@
+/*
+ * 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.assertFalse;
+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<byte[], byte[]> 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) {
+            @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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testPoisonOnFailure() throws IOException {
+        final AtomicInteger poisonCount = new AtomicInteger(0);
+
+        final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
+            @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<Object>() {
+            @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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testAllDelimitedMessagesSent() throws IOException {
+        final AtomicInteger poisonCount = new AtomicInteger(0);
+
+        final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger) {
+            @Override
+            protected void poison() {
+                poisonCount.incrementAndGet();
+                super.poison();
+            }
+        };
+
+        final AtomicInteger correctMessages = new AtomicInteger(0);
+        final AtomicInteger incorrectMessages = new AtomicInteger(0);
+        doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                final ProducerRecord<byte[], byte[]> 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.getFailedFlowFiles().contains(flowFile));
+        assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
+
+        assertEquals(7, correctMessages.get());
+        assertEquals(0, incorrectMessages.get());
+
+        verify(producer, times(1)).flush();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
new file mode 100644
index 0000000..7c70194
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import static org.junit.Assert.assertEquals;
+
+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<String, Object> 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);
+        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<String, Object> 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);
+        assertEquals(0, pool.available());
+
+        final PublisherLease lease = pool.obtainPublisher();
+        assertEquals(0, pool.available());
+
+        lease.poison();
+        lease.close();
+        assertEquals(0, pool.available());
+    }
+
+}


[16/20] nifi git commit: NIFI-2777: NIFI-2856: - Only performing response merging when the node is the cluster cooridinator even if there is a single response. - Fixing PropertyDescriptor merging to ensure the 'choosen' descriptor is included in map of a

Posted by jp...@apache.org.
NIFI-2777:
NIFI-2856:
- Only performing response merging when the node is the cluster cooridinator even if there is a single response.
- Fixing PropertyDescriptor merging to ensure the 'choosen' descriptor is included in map of all responses.

This closes #1095.


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 0b1d15a786a46dfb569197f97766e421b49096ac
Parents: 47b368f
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Oct 4 14:52:18 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:40:52 2016 -0500

----------------------------------------------------------------------
 .../coordination/http/HttpResponseMapper.java   |  65 ++++++
 .../coordination/http/HttpResponseMerger.java   |  65 ------
 .../http/StandardHttpResponseMapper.java        | 228 +++++++++++++++++++
 .../http/StandardHttpResponseMerger.java        | 227 ------------------
 .../StandardAsyncClusterResponse.java           |  22 +-
 .../ThreadPoolRequestReplicator.java            |  26 +--
 .../node/NodeClusterCoordinator.java            |  37 +--
 .../manager/PropertyDescriptorDtoMerger.java    |   2 +-
 .../http/StandardHttpResponseMapperSpec.groovy  | 217 ++++++++++++++++++
 .../http/StandardHttpResponseMergerSpec.groovy  | 218 ------------------
 10 files changed, 555 insertions(+), 552 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMapper.java
new file mode 100644
index 0000000..659f5e1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMapper.java
@@ -0,0 +1,65 @@
+/*
+ * 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.cluster.coordination.http;
+
+import org.apache.nifi.cluster.manager.NodeResponse;
+
+import java.net.URI;
+import java.util.Set;
+
+/**
+ * <p>
+ * An HttpResponseMapper is responsible for taking the responses from all nodes in a cluster
+ * and distilling them down to a single response that would be appropriate to respond with, to the
+ * user/client who made the original web requests.
+ * </p>
+ */
+public interface HttpResponseMapper {
+
+    /**
+     * Maps the responses from all nodes in the cluster to a single NodeResponse object that
+     * is appropriate to respond with
+     *
+     * @param uri the URI of the web request that was made
+     * @param httpMethod the HTTP Method that was used when making the request
+     * @param nodeResponses the responses received from the individual nodes
+     *
+     * @return a single NodeResponse that represents the response that should be returned to the user/client
+     */
+    NodeResponse mapResponses(URI uri, String httpMethod, Set<NodeResponse> nodeResponses, boolean merge);
+
+    /**
+     * Returns a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
+     * that indicate that the node was unable to fulfill the request
+     *
+     * @param allResponses the responses to filter
+     *
+     * @return a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
+     *         that indicate that the node was unable to fulfill the request
+     */
+    Set<NodeResponse> getProblematicNodeResponses(Set<NodeResponse> allResponses);
+
+    /**
+     * Indicates whether or not the responses from nodes for the given URI & HTTP method must be interpreted in order to merge them
+     *
+     * @param uri the URI of the request
+     * @param httpMethod the HTTP Method of the request
+     * @return <code>true</code> if the response must be interpreted, <code>false</code> otherwise
+     */
+    boolean isResponseInterpreted(URI uri, String httpMethod);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMerger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMerger.java
deleted file mode 100644
index 6102b74..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/HttpResponseMerger.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.cluster.coordination.http;
-
-import java.net.URI;
-import java.util.Set;
-
-import org.apache.nifi.cluster.manager.NodeResponse;
-
-/**
- * <p>
- * An HttpResponseMapper is responsible for taking the responses from all nodes in a cluster
- * and distilling them down to a single response that would be appropriate to respond with, to the
- * user/client who made the original web requests.
- * </p>
- */
-public interface HttpResponseMerger {
-
-    /**
-     * Maps the responses from all nodes in the cluster to a single NodeResponse object that
-     * is appropriate to respond with
-     *
-     * @param uri the URI of the web request that was made
-     * @param httpMethod the HTTP Method that was used when making the request
-     * @param nodeResponses the responses received from the individual nodes
-     *
-     * @return a single NodeResponse that represents the response that should be returned to the user/client
-     */
-    NodeResponse mergeResponses(URI uri, String httpMethod, Set<NodeResponse> nodeResponses);
-
-    /**
-     * Returns a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
-     * that indicate that the node was unable to fulfill the request
-     *
-     * @param allResponses the responses to filter
-     *
-     * @return a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
-     *         that indicate that the node was unable to fulfill the request
-     */
-    Set<NodeResponse> getProblematicNodeResponses(Set<NodeResponse> allResponses);
-
-    /**
-     * Indicates whether or not the responses from nodes for the given URI & HTTP method must be interpreted in order to merge them
-     *
-     * @param uri the URI of the request
-     * @param httpMethod the HTTP Method of the request
-     * @return <code>true</code> if the response must be interpreted, <code>false</code> otherwise
-     */
-    boolean isResponseInterpreted(URI uri, String httpMethod);
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
new file mode 100644
index 0000000..2f1bb18
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
@@ -0,0 +1,228 @@
+/*
+ * 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.cluster.coordination.http;
+
+import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpiontMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerBulletinsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerConfigurationEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceReferenceEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ControllerStatusEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.CurrentUserEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FlowConfigurationEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FunnelEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FunnelsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.InputPortsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.LabelEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.LabelsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.OutputPortsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.PortEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorStatusEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceEventEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceQueryEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupStatusEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.SystemDiagnosticsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.TemplatesEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.stream.io.NullOutputStream;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.StreamingOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class StandardHttpResponseMapper implements HttpResponseMapper {
+
+    private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMapper.class);
+
+    private final List<EndpointResponseMerger> endpointMergers = new ArrayList<>();
+
+    public StandardHttpResponseMapper(final NiFiProperties nifiProperties) {
+        final String snapshotFrequency = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
+        long snapshotMillis;
+        try {
+            snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
+        } catch (final Exception e) {
+            snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
+        }
+        endpointMergers.add(new ControllerStatusEndpointMerger());
+        endpointMergers.add(new ControllerBulletinsEndpointMerger());
+        endpointMergers.add(new GroupStatusEndpointMerger());
+        endpointMergers.add(new ProcessorStatusEndpointMerger());
+        endpointMergers.add(new ConnectionStatusEndpiontMerger());
+        endpointMergers.add(new PortStatusEndpointMerger());
+        endpointMergers.add(new RemoteProcessGroupStatusEndpointMerger());
+        endpointMergers.add(new ProcessorEndpointMerger());
+        endpointMergers.add(new ProcessorsEndpointMerger());
+        endpointMergers.add(new ConnectionEndpointMerger());
+        endpointMergers.add(new ConnectionsEndpointMerger());
+        endpointMergers.add(new PortEndpointMerger());
+        endpointMergers.add(new InputPortsEndpointMerger());
+        endpointMergers.add(new OutputPortsEndpointMerger());
+        endpointMergers.add(new RemoteProcessGroupEndpointMerger());
+        endpointMergers.add(new RemoteProcessGroupsEndpointMerger());
+        endpointMergers.add(new ProcessGroupEndpointMerger());
+        endpointMergers.add(new ProcessGroupsEndpointMerger());
+        endpointMergers.add(new FlowSnippetEndpointMerger());
+        endpointMergers.add(new ProvenanceQueryEndpointMerger());
+        endpointMergers.add(new ProvenanceEventEndpointMerger());
+        endpointMergers.add(new ControllerServiceEndpointMerger());
+        endpointMergers.add(new ControllerServicesEndpointMerger());
+        endpointMergers.add(new ControllerServiceReferenceEndpointMerger());
+        endpointMergers.add(new ReportingTaskEndpointMerger());
+        endpointMergers.add(new ReportingTasksEndpointMerger());
+        endpointMergers.add(new DropRequestEndpiontMerger());
+        endpointMergers.add(new ListFlowFilesEndpointMerger());
+        endpointMergers.add(new ComponentStateEndpointMerger());
+        endpointMergers.add(new BulletinBoardEndpointMerger());
+        endpointMergers.add(new StatusHistoryEndpointMerger(snapshotMillis));
+        endpointMergers.add(new SystemDiagnosticsEndpointMerger());
+        endpointMergers.add(new CountersEndpointMerger());
+        endpointMergers.add(new FlowMerger());
+        endpointMergers.add(new ControllerConfigurationEndpointMerger());
+        endpointMergers.add(new CurrentUserEndpointMerger());
+        endpointMergers.add(new FlowConfigurationEndpointMerger());
+        endpointMergers.add(new TemplatesEndpointMerger());
+        endpointMergers.add(new LabelEndpointMerger());
+        endpointMergers.add(new LabelsEndpointMerger());
+        endpointMergers.add(new FunnelEndpointMerger());
+        endpointMergers.add(new FunnelsEndpointMerger());
+    }
+
+    @Override
+    public NodeResponse mapResponses(final URI uri, final String httpMethod, final Set<NodeResponse> nodeResponses, final boolean merge) {
+        final boolean hasSuccess = hasSuccessfulResponse(nodeResponses);
+        if (!hasSuccess) {
+            // If we have a response that is a 3xx, 4xx, or 5xx, then we want to choose that.
+            // Otherwise, it doesn't matter which one we choose. We do this because if we replicate
+            // a mutable request, it's possible that one node will respond with a 409, for instance, while
+            // others respond with a 150-Continue. We do not want to pick the 150-Continue; instead, we want
+            // the failed response.
+            final NodeResponse clientResponse = nodeResponses.stream().filter(p -> p.getStatus() > 299).findAny().orElse(nodeResponses.iterator().next());
+
+            // Drain the response from all nodes except for the 'chosen one'. This ensures that we don't
+            // leave data lingering on the socket and ensures that we don't consume the content of the response
+            // that we intend to respond with
+            drainResponses(nodeResponses, clientResponse);
+            return clientResponse;
+        }
+
+        // Determine which responses are successful
+        final Set<NodeResponse> successResponses = nodeResponses.stream().filter(p -> p.is2xx()).collect(Collectors.toSet());
+        final Set<NodeResponse> problematicResponses = nodeResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
+
+        final NodeResponse clientResponse;
+        if ("GET".equalsIgnoreCase(httpMethod) && problematicResponses.size() > 0) {
+            // If there are problematic responses, at least one of the nodes couldn't complete the request
+            clientResponse = problematicResponses.stream().filter(p -> p.getStatus() >= 400 && p.getStatus() < 500).findFirst().orElse(
+                    problematicResponses.stream().filter(p -> p.getStatus() > 500).findFirst().orElse(problematicResponses.iterator().next()));
+            return clientResponse;
+        } else {
+            // Choose any of the successful responses to be the 'chosen one'.
+            clientResponse = successResponses.iterator().next();
+        }
+
+        if (merge == false) {
+            return clientResponse;
+        }
+
+        EndpointResponseMerger merger = getEndpointResponseMerger(uri, httpMethod);
+        if (merger == null) {
+            return clientResponse;
+        }
+
+        final NodeResponse response = merger.merge(uri, httpMethod, successResponses, problematicResponses, clientResponse);
+        return response;
+    }
+
+    @Override
+    public Set<NodeResponse> getProblematicNodeResponses(final Set<NodeResponse> allResponses) {
+        // Check if there are any 2xx responses
+        final boolean containsSuccessfulResponse = hasSuccessfulResponse(allResponses);
+
+        if (containsSuccessfulResponse) {
+            // If there is a 2xx response, we consider a response to be problematic if it is not 2xx
+            return allResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
+        } else {
+            // If no node is successful, we consider a problematic response to be only those that are 5xx
+            return allResponses.stream().filter(p -> p.is5xx()).collect(Collectors.toSet());
+        }
+    }
+
+    @Override
+    public boolean isResponseInterpreted(final URI uri, final String httpMethod) {
+        return getEndpointResponseMerger(uri, httpMethod) != null;
+    }
+
+    private EndpointResponseMerger getEndpointResponseMerger(final URI uri, final String httpMethod) {
+        return endpointMergers.stream().filter(p -> p.canHandle(uri, httpMethod)).findFirst().orElse(null);
+    }
+
+    private boolean hasSuccessfulResponse(final Set<NodeResponse> allResponses) {
+        return allResponses.stream().anyMatch(p -> p.is2xx());
+    }
+
+    private void drainResponses(final Set<NodeResponse> responses, final NodeResponse exclude) {
+        responses.stream()
+                .parallel() // parallelize the draining of the responses, since we have multiple streams to consume
+                .filter(response -> response != exclude) // don't include the explicitly excluded node
+                .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
+                .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
+    }
+
+    private void drainResponse(final NodeResponse response) {
+        if (response.hasThrowable()) {
+            return;
+        }
+
+        try {
+            ((StreamingOutput) response.getResponse().getEntity()).write(new NullOutputStream());
+        } catch (final IOException ioe) {
+            logger.info("Failed clearing out non-client response buffer from " + response.getNodeId() + " due to: " + ioe, ioe);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
deleted file mode 100644
index 7b1d8f6..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.coordination.http;
-
-import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpiontMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerBulletinsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerConfigurationEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceReferenceEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ControllerStatusEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.CurrentUserEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.FlowConfigurationEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.FunnelEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.FunnelsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.InputPortsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.LabelEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.LabelsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.OutputPortsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.PortEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorStatusEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceEventEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceQueryEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupStatusEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.SystemDiagnosticsEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.endpoints.TemplatesEndpointMerger;
-import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.stream.io.NullOutputStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.ws.rs.core.StreamingOutput;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-public class StandardHttpResponseMerger implements HttpResponseMerger {
-
-    private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class);
-
-    private final List<EndpointResponseMerger> endpointMergers = new ArrayList<>();
-
-    public StandardHttpResponseMerger(final NiFiProperties nifiProperties) {
-        final String snapshotFrequency = nifiProperties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
-        long snapshotMillis;
-        try {
-            snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
-        } catch (final Exception e) {
-            snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
-        }
-        endpointMergers.add(new ControllerStatusEndpointMerger());
-        endpointMergers.add(new ControllerBulletinsEndpointMerger());
-        endpointMergers.add(new GroupStatusEndpointMerger());
-        endpointMergers.add(new ProcessorStatusEndpointMerger());
-        endpointMergers.add(new ConnectionStatusEndpiontMerger());
-        endpointMergers.add(new PortStatusEndpointMerger());
-        endpointMergers.add(new RemoteProcessGroupStatusEndpointMerger());
-        endpointMergers.add(new ProcessorEndpointMerger());
-        endpointMergers.add(new ProcessorsEndpointMerger());
-        endpointMergers.add(new ConnectionEndpointMerger());
-        endpointMergers.add(new ConnectionsEndpointMerger());
-        endpointMergers.add(new PortEndpointMerger());
-        endpointMergers.add(new InputPortsEndpointMerger());
-        endpointMergers.add(new OutputPortsEndpointMerger());
-        endpointMergers.add(new RemoteProcessGroupEndpointMerger());
-        endpointMergers.add(new RemoteProcessGroupsEndpointMerger());
-        endpointMergers.add(new ProcessGroupEndpointMerger());
-        endpointMergers.add(new ProcessGroupsEndpointMerger());
-        endpointMergers.add(new FlowSnippetEndpointMerger());
-        endpointMergers.add(new ProvenanceQueryEndpointMerger());
-        endpointMergers.add(new ProvenanceEventEndpointMerger());
-        endpointMergers.add(new ControllerServiceEndpointMerger());
-        endpointMergers.add(new ControllerServicesEndpointMerger());
-        endpointMergers.add(new ControllerServiceReferenceEndpointMerger());
-        endpointMergers.add(new ReportingTaskEndpointMerger());
-        endpointMergers.add(new ReportingTasksEndpointMerger());
-        endpointMergers.add(new DropRequestEndpiontMerger());
-        endpointMergers.add(new ListFlowFilesEndpointMerger());
-        endpointMergers.add(new ComponentStateEndpointMerger());
-        endpointMergers.add(new BulletinBoardEndpointMerger());
-        endpointMergers.add(new StatusHistoryEndpointMerger(snapshotMillis));
-        endpointMergers.add(new SystemDiagnosticsEndpointMerger());
-        endpointMergers.add(new CountersEndpointMerger());
-        endpointMergers.add(new FlowMerger());
-        endpointMergers.add(new ControllerConfigurationEndpointMerger());
-        endpointMergers.add(new CurrentUserEndpointMerger());
-        endpointMergers.add(new FlowConfigurationEndpointMerger());
-        endpointMergers.add(new TemplatesEndpointMerger());
-        endpointMergers.add(new LabelEndpointMerger());
-        endpointMergers.add(new LabelsEndpointMerger());
-        endpointMergers.add(new FunnelEndpointMerger());
-        endpointMergers.add(new FunnelsEndpointMerger());
-    }
-
-    @Override
-    public NodeResponse mergeResponses(final URI uri, final String httpMethod, final Set<NodeResponse> nodeResponses) {
-        if (nodeResponses.size() == 1) {
-            return nodeResponses.iterator().next();
-        }
-
-        final boolean hasSuccess = hasSuccessfulResponse(nodeResponses);
-        if (!hasSuccess) {
-            // If we have a response that is a 3xx, 4xx, or 5xx, then we want to choose that.
-            // Otherwise, it doesn't matter which one we choose. We do this because if we replicate
-            // a mutable request, it's possible that one node will respond with a 409, for instance, while
-            // others respond with a 150-Continue. We do not want to pick the 150-Continue; instead, we want
-            // the failed response.
-            final NodeResponse clientResponse = nodeResponses.stream().filter(p -> p.getStatus() > 299).findAny().orElse(nodeResponses.iterator().next());
-
-            // Drain the response from all nodes except for the 'chosen one'. This ensures that we don't
-            // leave data lingering on the socket and ensures that we don't consume the content of the response
-            // that we intend to respond with
-            drainResponses(nodeResponses, clientResponse);
-            return clientResponse;
-        }
-
-        // Determine which responses are successful
-        final Set<NodeResponse> successResponses = nodeResponses.stream().filter(p -> p.is2xx()).collect(Collectors.toSet());
-        final Set<NodeResponse> problematicResponses = nodeResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
-
-        final NodeResponse clientResponse;
-        if ("GET".equalsIgnoreCase(httpMethod) && problematicResponses.size() > 0) {
-            // If there are problematic responses, at least one of the nodes couldn't complete the request
-            clientResponse = problematicResponses.stream().filter(p -> p.getStatus() >= 400 && p.getStatus() < 500).findFirst().orElse(
-                    problematicResponses.stream().filter(p -> p.getStatus() > 500).findFirst().orElse(problematicResponses.iterator().next()));
-            return clientResponse;
-        } else {
-            // Choose any of the successful responses to be the 'chosen one'.
-            clientResponse = successResponses.iterator().next();
-        }
-        EndpointResponseMerger merger = getEndpointResponseMerger(uri, httpMethod);
-        if (merger == null) {
-            return clientResponse;
-        }
-
-        final NodeResponse response = merger.merge(uri, httpMethod, successResponses, problematicResponses, clientResponse);
-        return response;
-    }
-
-    @Override
-    public Set<NodeResponse> getProblematicNodeResponses(final Set<NodeResponse> allResponses) {
-        // Check if there are any 2xx responses
-        final boolean containsSuccessfulResponse = hasSuccessfulResponse(allResponses);
-
-        if (containsSuccessfulResponse) {
-            // If there is a 2xx response, we consider a response to be problematic if it is not 2xx
-            return allResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
-        } else {
-            // If no node is successful, we consider a problematic response to be only those that are 5xx
-            return allResponses.stream().filter(p -> p.is5xx()).collect(Collectors.toSet());
-        }
-    }
-
-    @Override
-    public boolean isResponseInterpreted(final URI uri, final String httpMethod) {
-        return getEndpointResponseMerger(uri, httpMethod) != null;
-    }
-
-    private EndpointResponseMerger getEndpointResponseMerger(final URI uri, final String httpMethod) {
-        return endpointMergers.stream().filter(p -> p.canHandle(uri, httpMethod)).findFirst().orElse(null);
-    }
-
-    private boolean hasSuccessfulResponse(final Set<NodeResponse> allResponses) {
-        return allResponses.stream().anyMatch(p -> p.is2xx());
-    }
-
-    private void drainResponses(final Set<NodeResponse> responses, final NodeResponse exclude) {
-        responses.stream()
-                .parallel() // parallelize the draining of the responses, since we have multiple streams to consume
-                .filter(response -> response != exclude) // don't include the explicitly excluded node
-                .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
-                .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
-    }
-
-    private void drainResponse(final NodeResponse response) {
-        if (response.hasThrowable()) {
-            return;
-        }
-
-        try {
-            ((StreamingOutput) response.getResponse().getEntity()).write(new NullOutputStream());
-        } catch (final IOException ioe) {
-            logger.info("Failed clearing out non-client response buffer from " + response.getNodeId() + " due to: " + ioe, ioe);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/StandardAsyncClusterResponse.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/StandardAsyncClusterResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/StandardAsyncClusterResponse.java
index 3bcc8e7..926151e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/StandardAsyncClusterResponse.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/StandardAsyncClusterResponse.java
@@ -17,6 +17,12 @@
 
 package org.apache.nifi.cluster.coordination.http.replication;
 
+import org.apache.nifi.cluster.coordination.http.HttpResponseMapper;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.net.URI;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,12 +33,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
-import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
-import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class StandardAsyncClusterResponse implements AsyncClusterResponse {
     private static final Logger logger = LoggerFactory.getLogger(StandardAsyncClusterResponse.class);
 
@@ -40,10 +40,11 @@ public class StandardAsyncClusterResponse implements AsyncClusterResponse {
     private final Set<NodeIdentifier> nodeIds;
     private final URI uri;
     private final String method;
-    private final HttpResponseMerger responseMerger;
+    private final HttpResponseMapper responseMapper;
     private final CompletionCallback completionCallback;
     private final Runnable completedResultFetchedCallback;
     private final long creationTimeNanos;
+    private final boolean merge;
 
     private final Map<NodeIdentifier, ResponseHolder> responseMap = new HashMap<>();
     private final AtomicInteger requestsCompleted = new AtomicInteger(0);
@@ -52,18 +53,19 @@ public class StandardAsyncClusterResponse implements AsyncClusterResponse {
     private RuntimeException failure; // guarded by synchronizing on this
 
     public StandardAsyncClusterResponse(final String id, final URI uri, final String method, final Set<NodeIdentifier> nodeIds,
-        final HttpResponseMerger responseMerger, final CompletionCallback completionCallback, final Runnable completedResultFetchedCallback) {
+                                        final HttpResponseMapper responseMapper, final CompletionCallback completionCallback, final Runnable completedResultFetchedCallback, final boolean merge) {
         this.id = id;
         this.nodeIds = Collections.unmodifiableSet(new HashSet<>(nodeIds));
         this.uri = uri;
         this.method = method;
+        this.merge = merge;
 
         creationTimeNanos = System.nanoTime();
         for (final NodeIdentifier nodeId : nodeIds) {
             responseMap.put(nodeId, new ResponseHolder(creationTimeNanos));
         }
 
-        this.responseMerger = responseMerger;
+        this.responseMapper = responseMapper;
         this.completionCallback = completionCallback;
         this.completedResultFetchedCallback = completedResultFetchedCallback;
     }
@@ -142,7 +144,7 @@ public class StandardAsyncClusterResponse implements AsyncClusterResponse {
             .map(p -> p.getResponse())
             .filter(response -> response != null)
             .collect(Collectors.toSet());
-        mergedResponse = responseMerger.mergeResponses(uri, method, nodeResponses);
+        mergedResponse = responseMapper.mapResponses(uri, method, nodeResponses, merge);
 
         logger.debug("Notifying all that merged response is complete for {}", id);
         this.notifyAll();

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
index c1ee77b..258588d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
@@ -27,8 +27,8 @@ import org.apache.nifi.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
-import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
-import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
+import org.apache.nifi.cluster.coordination.http.HttpResponseMapper;
+import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMapper;
 import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
 import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
 import org.apache.nifi.cluster.manager.NodeResponse;
@@ -85,7 +85,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
     private final Client client; // the client to use for issuing requests
     private final int connectionTimeoutMs; // connection timeout per node request
     private final int readTimeoutMs; // read timeout per node request
-    private final HttpResponseMerger responseMerger;
+    private final HttpResponseMapper responseMapper;
     private final EventReporter eventReporter;
     private final RequestCompletionCallback callback;
     private final ClusterCoordinator clusterCoordinator;
@@ -140,7 +140,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         this.clusterCoordinator = clusterCoordinator;
         this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
         this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
-        this.responseMerger = new StandardHttpResponseMerger(nifiProperties);
+        this.responseMapper = new StandardHttpResponseMapper(nifiProperties);
         this.eventReporter = eventReporter;
         this.callback = callback;
 
@@ -249,12 +249,12 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
             lock.lock();
             try {
                 logger.debug("Lock {} obtained in order to replicate request {} {}", method, uri);
-                return replicate(nodeIds, method, uri, entity, updatedHeaders, performVerification, null, !performVerification);
+                return replicate(nodeIds, method, uri, entity, updatedHeaders, performVerification, null, !performVerification, true);
             } finally {
                 lock.unlock();
             }
         } else {
-            return replicate(nodeIds, method, uri, entity, updatedHeaders, performVerification, null, !performVerification);
+            return replicate(nodeIds, method, uri, entity, updatedHeaders, performVerification, null, !performVerification, true);
         }
     }
 
@@ -269,7 +269,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
             updatedHeaders.put(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN, proxiedEntitiesChain);
         }
 
-        return replicate(Collections.singleton(coordinatorNodeId), method, uri, entity, updatedHeaders, false, null, false);
+        return replicate(Collections.singleton(coordinatorNodeId), method, uri, entity, updatedHeaders, false, null, false, false);
     }
 
     /**
@@ -286,7 +286,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
      * @return an AsyncClusterResponse that can be used to obtain the result
      */
     private AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers, boolean performVerification,
-                                           StandardAsyncClusterResponse response, boolean executionPhase) {
+                                           StandardAsyncClusterResponse response, boolean executionPhase, boolean merge) {
 
         // state validation
         Objects.requireNonNull(nodeIds);
@@ -344,7 +344,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         // create a response object if one was not already passed to us
         if (response == null) {
             response = new StandardAsyncClusterResponse(requestId, uri, method, nodeIds,
-                    responseMerger, completionCallback, responseConsumedCallback);
+                    responseMapper, completionCallback, responseConsumedCallback, merge);
             responseMap.put(requestId, response);
         }
 
@@ -358,7 +358,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         final boolean mutableRequest = isMutableRequest(method, uri.getPath());
         if (mutableRequest && performVerification) {
             logger.debug("Performing verification (first phase of two-phase commit) for Request ID {}", requestId);
-            performVerification(nodeIds, method, uri, entity, updatedHeaders, response);
+            performVerification(nodeIds, method, uri, entity, updatedHeaders, response, merge);
             return response;
         }
 
@@ -383,7 +383,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
     }
 
 
-    private void performVerification(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers, StandardAsyncClusterResponse clusterResponse) {
+    private void performVerification(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers, StandardAsyncClusterResponse clusterResponse, boolean merge) {
         logger.debug("Verifying that mutable request {} {} can be made", method, uri.getPath());
 
         final Map<String, String> validationHeaders = new HashMap<>(headers);
@@ -418,7 +418,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
                         // to all nodes and we are finished.
                         if (dissentingCount == 0) {
                             logger.debug("Received verification from all {} nodes that mutable request {} {} can be made", numNodes, method, uri.getPath());
-                            replicate(nodeIds, method, uri, entity, headers, false, clusterResponse, true);
+                            replicate(nodeIds, method, uri, entity, headers, false, clusterResponse, true, merge);
                             return;
                         }
 
@@ -743,7 +743,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
             // create the resource
             WebResource resource = client.resource(uri);
 
-            if (responseMerger.isResponseInterpreted(uri, method)) {
+            if (responseMapper.isResponseInterpreted(uri, method)) {
                 resource.addFilter(new GZIPContentEncodingFilter(false));
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
index e50d8fa..a17e0de 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
@@ -16,26 +16,11 @@
  */
 package org.apache.nifi.cluster.coordination.node;
 
-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.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
 import org.apache.commons.collections4.queue.CircularFifoQueue;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
-import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
-import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
+import org.apache.nifi.cluster.coordination.http.HttpResponseMapper;
+import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMapper;
 import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback;
 import org.apache.nifi.cluster.event.Event;
 import org.apache.nifi.cluster.event.NodeEvent;
@@ -69,6 +54,22 @@ import org.apache.nifi.web.revision.RevisionManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+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.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
 public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandler, RequestCompletionCallback {
 
     private static final Logger logger = LoggerFactory.getLogger(NodeClusterCoordinator.class);
@@ -900,7 +901,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
          * state even if they had problems handling the request.
          */
         if (mutableRequest) {
-            final HttpResponseMerger responseMerger = new StandardHttpResponseMerger(nifiProperties);
+            final HttpResponseMapper responseMerger = new StandardHttpResponseMapper(nifiProperties);
             final Set<NodeResponse> problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses);
 
             // all nodes failed

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/PropertyDescriptorDtoMerger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/PropertyDescriptorDtoMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/PropertyDescriptorDtoMerger.java
index e7ab881..3c18ced 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/PropertyDescriptorDtoMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/PropertyDescriptorDtoMerger.java
@@ -33,7 +33,7 @@ public class PropertyDescriptorDtoMerger {
         for (final Map.Entry<NodeIdentifier, PropertyDescriptorDTO> nodeEntry : dtoMap.entrySet()) {
             final PropertyDescriptorDTO nodePropertyDescriptor = nodeEntry.getValue();
             final List<AllowableValueEntity> nodePropertyDescriptorAllowableValues = nodePropertyDescriptor.getAllowableValues();
-            if (clientPropertyDescriptor != nodePropertyDescriptor && nodePropertyDescriptorAllowableValues != null) {
+            if (nodePropertyDescriptorAllowableValues != null) {
                 nodePropertyDescriptorAllowableValues.stream().forEach(allowableValueEntity -> {
                     allowableValueMap.computeIfAbsent(nodePropertyDescriptorAllowableValues.indexOf(allowableValueEntity), propertyDescriptorToAllowableValue -> new ArrayList<>())
                             .add(allowableValueEntity);

http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapperSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapperSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapperSpec.groovy
new file mode 100644
index 0000000..243fd1a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapperSpec.groovy
@@ -0,0 +1,217 @@
+/*
+ * 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.cluster.coordination.http
+
+import com.sun.jersey.api.client.ClientResponse
+import org.apache.nifi.cluster.manager.NodeResponse
+import org.apache.nifi.cluster.protocol.NodeIdentifier
+import org.apache.nifi.util.NiFiProperties
+import org.apache.nifi.web.api.dto.ConnectionDTO
+import org.apache.nifi.web.api.dto.ControllerConfigurationDTO
+import org.apache.nifi.web.api.dto.FunnelDTO
+import org.apache.nifi.web.api.dto.LabelDTO
+import org.apache.nifi.web.api.dto.PermissionsDTO
+import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO
+import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO
+import org.apache.nifi.web.api.entity.ConnectionEntity
+import org.apache.nifi.web.api.entity.ConnectionsEntity
+import org.apache.nifi.web.api.entity.ControllerConfigurationEntity
+import org.apache.nifi.web.api.entity.FunnelEntity
+import org.apache.nifi.web.api.entity.FunnelsEntity
+import org.apache.nifi.web.api.entity.LabelEntity
+import org.apache.nifi.web.api.entity.LabelsEntity
+import org.codehaus.jackson.map.ObjectMapper
+import org.codehaus.jackson.map.SerializationConfig
+import org.codehaus.jackson.map.annotate.JsonSerialize
+import org.codehaus.jackson.xc.JaxbAnnotationIntrospector
+import spock.lang.Specification
+import spock.lang.Unroll
+
+@Unroll
+class StandardHttpResponseMapperSpec extends Specification {
+
+    def setup() {
+        def propFile = StandardHttpResponseMapperSpec.class.getResource("/conf/nifi.properties").getFile()
+        System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
+    }
+
+    def cleanup() {
+        System.clearProperty NiFiProperties.PROPERTIES_FILE_PATH
+    }
+
+    def "MergeResponses: mixed HTTP GET response statuses, expecting #expectedStatus"() {
+        given:
+        def responseMapper = new StandardHttpResponseMapper(NiFiProperties.createBasicNiFiProperties(null,null))
+        def requestUri = new URI('http://server/resource')
+        def requestId = UUID.randomUUID().toString()
+        def Map<ClientResponse, Map<String, Integer>> mockToRequestEntity = [:]
+        def nodeResponseSet = nodeResponseData.collect {
+            int n = it.node
+            def clientResponse = Mock(ClientResponse)
+            mockToRequestEntity.put clientResponse, it
+            new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, clientResponse, 500L, requestId)
+        } as Set
+
+        when:
+        def returnedResponse = responseMapper.mapResponses(requestUri, 'get', nodeResponseSet, true).getStatus()
+
+        then:
+        mockToRequestEntity.entrySet().forEach {
+            ClientResponse mockClientResponse = it.key
+            _ * mockClientResponse.getStatus() >> it.value.status
+        }
+        0 * _
+        returnedResponse == expectedStatus
+
+        where:
+        nodeResponseData                                                                || expectedStatus
+        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 401]] as Set || 401
+        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 403]] as Set || 403
+        [[node: 1, status: 200], [node: 2, status: 403], [node: 3, status: 500]] as Set || 403
+        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 500]] as Set || 500
+    }
+
+    def "MergeResponses: #responseEntities.size() HTTP 200 #httpMethod responses for #requestUriPart"() {
+        given: "json serialization setup"
+        def mapper = new ObjectMapper();
+        def jaxbIntrospector = new JaxbAnnotationIntrospector();
+        def SerializationConfig serializationConfig = mapper.getSerializationConfig();
+        mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(JsonSerialize.Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector));
+
+        and: "setup of the data to be used in the test"
+        def responseMerger = new StandardHttpResponseMapper(NiFiProperties.createBasicNiFiProperties(null,null))
+        def requestUri = new URI("http://server/$requestUriPart")
+        def requestId = UUID.randomUUID().toString()
+        def Map<ClientResponse, Object> mockToRequestEntity = [:]
+        def n = 0
+        def nodeResponseSet = responseEntities.collect {
+            ++n
+            def clientResponse = Mock(ClientResponse)
+            mockToRequestEntity.put clientResponse, it
+            new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, clientResponse, 500L, requestId)
+        } as Set
+
+        when:
+        def returnedResponse = responseMerger.mapResponses(requestUri, httpMethod, nodeResponseSet, true)
+
+        then:
+        mockToRequestEntity.entrySet().forEach {
+            ClientResponse mockClientResponse = it.key
+            def entity = it.value
+            _ * mockClientResponse.getStatus() >> 200
+            1 * mockClientResponse.getEntity(_) >> entity
+        }
+        responseEntities.size() == mockToRequestEntity.size()
+        0 * _
+        def returnedJson = mapper.writeValueAsString(returnedResponse.getUpdatedEntity())
+        def expectedJson = mapper.writeValueAsString(expectedEntity)
+        returnedJson == expectedJson
+
+        where:
+        requestUriPart                                             | httpMethod | responseEntities                                                                                     ||
+                expectedEntity
+        'nifi-api/controller/config'                               | 'get'      | [
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))]                                                      ||
+                // expectedEntity
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))
+        'nifi-api/controller/config'                               | 'put'      | [
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))]                                                      ||
+                // expectedEntity
+                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
+                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))
+        "nifi-api/process-groups/${UUID.randomUUID()}/connections" | 'get'      | [
+                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status: new
+                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())] as Set),
+                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status: new
+                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 100)))] as Set),
+                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status: new
+                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 500)), component: new ConnectionDTO())] as Set)] ||
+                // expectedEntity
+                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
+                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 900,
+                                input: '0 (900 bytes)', output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))] as Set)
+        "nifi-api/process-groups/${UUID.randomUUID()}/connections" | 'post'     | [
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO()),
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300))),
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())]      ||
+                // expectedEntity
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
+                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 900, input: '0 (900 bytes)',
+                                output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))
+        "nifi-api/connections/${UUID.randomUUID()}"                | 'get'      | [
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 400)), component: new ConnectionDTO()),
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300))),
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
+                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())]      ||
+                // expectedEntity
+                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
+                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 1000,
+                                input: '0 (1,000 bytes)', output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))
+        "nifi-api/process-groups/${UUID.randomUUID()}/labels" | 'get'      | [
+                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())] as Set),
+                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set),
+                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())] as Set)] ||
+                // expectedEntity
+                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set)
+        "nifi-api/process-groups/${UUID.randomUUID()}/labels" | 'post'     | [
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO()),
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())]      ||
+                // expectedEntity
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
+        "nifi-api/labels/${UUID.randomUUID()}"                | 'get'      | [
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO()),
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())]      ||
+                // expectedEntity
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
+        "nifi-api/process-groups/${UUID.randomUUID()}/funnels" | 'get'      | [
+                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())] as Set),
+                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set),
+                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())] as Set)] ||
+                // expectedEntity
+                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set)
+        "nifi-api/process-groups/${UUID.randomUUID()}/funnels" | 'post'     | [
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO()),
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())]      ||
+                // expectedEntity
+                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
+        "nifi-api/funnels/${UUID.randomUUID()}"                | 'get'      | [
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO()),
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())]      ||
+                // expectedEntity
+                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
+    }
+}


[10/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
index a85563d..6fd9053 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
@@ -16,104 +16,36 @@
  */
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.UUID;
+import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
 
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import org.junit.Before;
+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;
 
 public class ConsumeKafkaTest {
 
-    static class MockConsumerPool extends ConsumerPool {
-
-        final int actualMaxLeases;
-        final List<String> actualTopics;
-        final Map<String, String> actualKafkaProperties;
-        boolean throwKafkaExceptionOnPoll = false;
-        boolean throwKafkaExceptionOnCommit = false;
-        Queue<ConsumerRecords<byte[], byte[]>> nextPlannedRecordsQueue = new ArrayDeque<>();
-        Map<TopicPartition, OffsetAndMetadata> nextExpectedCommitOffsets = null;
-        Map<TopicPartition, OffsetAndMetadata> actualCommitOffsets = null;
-        boolean wasConsumerLeasePoisoned = false;
-        boolean wasConsumerLeaseClosed = false;
-        boolean wasPoolClosed = false;
-
-        public MockConsumerPool(int maxLeases, List<String> topics, Map<String, String> kafkaProperties, ComponentLog logger) {
-            super(maxLeases, topics, kafkaProperties, null);
-            actualMaxLeases = maxLeases;
-            actualTopics = topics;
-            actualKafkaProperties = kafkaProperties;
-        }
-
-        @Override
-        public ConsumerLease obtainConsumer() {
-            return new ConsumerLease() {
-                @Override
-                public ConsumerRecords<byte[], byte[]> poll() {
-                    if (throwKafkaExceptionOnPoll) {
-                        throw new KafkaException("i planned to fail");
-                    }
-                    final ConsumerRecords<byte[], byte[]> records = nextPlannedRecordsQueue.poll();
-                    return (records == null) ? ConsumerRecords.empty() : records;
-                }
-
-                @Override
-                public void commitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) {
-                    if (throwKafkaExceptionOnCommit) {
-                        throw new KafkaException("i planned to fail");
-                    }
-                    actualCommitOffsets = offsets;
-                }
-
-                @Override
-                public void poison() {
-                    wasConsumerLeasePoisoned = true;
-                }
-
-                @Override
-                public void close() {
-                    wasConsumerLeaseClosed = true;
-                }
-            };
-        }
-
-        @Override
-        public void close() {
-            wasPoolClosed = true;
-        }
-
-        void resetState() {
-            throwKafkaExceptionOnPoll = false;
-            throwKafkaExceptionOnCommit = false;
-            nextPlannedRecordsQueue = null;
-            nextExpectedCommitOffsets = null;
-            wasConsumerLeasePoisoned = false;
-            wasConsumerLeaseClosed = false;
-            wasPoolClosed = false;
-        }
+    Consumer<byte[], byte[]> mockConsumer = null;
+    ConsumerLease mockLease = null;
+    ConsumerPool mockConsumerPool = null;
 
+    @Before
+    public void setup() {
+        mockConsumer = mock(Consumer.class);
+        mockLease = mock(ConsumerLease.class);
+        mockConsumerPool = mock(ConsumerPool.class);
     }
 
     @Test
@@ -174,31 +106,14 @@ public class ConsumeKafkaTest {
     public void validateGetAllMessages() throws Exception {
         String groupName = "validateGetAllMessages";
 
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
-
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-        mockPool.nextPlannedRecordsQueue.add(secondRecs);
+        when(mockConsumerPool.obtainConsumer(anyObject())).thenReturn(mockLease);
+        when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE);
+        when(mockLease.commit()).thenReturn(Boolean.TRUE);
 
         ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
             @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
+            protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
+                return mockConsumerPool;
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
@@ -207,69 +122,29 @@ public class ConsumeKafkaTest {
         runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-
         runner.run(1, false);
 
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        if (mockPool.nextPlannedRecordsQueue.isEmpty()) {
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4")).count());
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-5")).count());
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-6")).count());
-            assertEquals(2, mockPool.actualCommitOffsets.size());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset());
-        } else {
-            assertEquals(2, mockPool.actualCommitOffsets.size());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-        }
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
+        verify(mockConsumerPool, times(1)).obtainConsumer(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 validateGetLotsOfMessages() throws Exception {
-        String groupName = "validateGetLotsOfMessages";
-
-        final byte[][] firstPassValues = new byte[10010][1];
-        for (final byte[] value : firstPassValues) {
-            value[0] = 0x12;
-        }
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
+    public void validateGetErrorMessages() throws Exception {
+        String groupName = "validateGetErrorMessages";
 
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-        mockPool.nextPlannedRecordsQueue.add(secondRecs);
+        when(mockConsumerPool.obtainConsumer(anyObject())).thenReturn(mockLease);
+        when(mockLease.continuePolling()).thenReturn(true, false);
+        when(mockLease.commit()).thenReturn(Boolean.FALSE);
 
         ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
             @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
+            protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
+                return mockConsumerPool;
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
@@ -278,352 +153,15 @@ public class ConsumeKafkaTest {
         runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-
         runner.run(1, false);
 
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(10010, flowFiles.stream().map(ff -> ff.toByteArray()).filter(content -> content.length == 1 && content[0] == 0x12).count());
-        assertEquals(1, mockPool.nextPlannedRecordsQueue.size());
-
-        assertEquals(1, mockPool.actualCommitOffsets.size());
-        assertEquals(10011L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
+        verify(mockConsumerPool, times(1)).obtainConsumer(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);
     }
 
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        final TopicPartition tPart = new TopicPartition(topic, partition);
-        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
-        long offset = startingOffset;
-        for (final byte[] rawRecord : rawRecords) {
-            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord);
-            records.add(rec);
-        }
-        map.put(tPart, records);
-        return new ConsumerRecords(map);
-    }
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final Map<byte[], byte[]> rawRecords) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        final TopicPartition tPart = new TopicPartition(topic, partition);
-        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
-        long offset = startingOffset;
-
-        for (final Map.Entry<byte[], byte[]> entry : rawRecords.entrySet()) {
-            final byte[] key = entry.getKey();
-            final byte[] rawRecord = entry.getValue();
-            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, key, rawRecord);
-            records.add(rec);
-        }
-        map.put(tPart, records);
-        return new ConsumerRecords(map);
-    }
-
-    private ConsumerRecords<byte[], byte[]> mergeRecords(final ConsumerRecords<byte[], byte[]>... records) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        for (final ConsumerRecords<byte[], byte[]> rec : records) {
-            rec.partitions().stream().forEach((part) -> {
-                final List<ConsumerRecord<byte[], byte[]>> conRecs = rec.records(part);
-                if (map.get(part) != null) {
-                    throw new IllegalStateException("already have that topic/partition in the record map");
-                }
-                map.put(part, conRecs);
-            });
-        }
-        return new ConsumerRecords<>(map);
-    }
-
-    @Test
-    public void validateGetAllMessagesWithProvidedDemarcator() throws Exception {
-        String groupName = "validateGetAllMessagesWithProvidedDemarcator";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues),
-                createConsumerRecords("bar", 1, 1L, secondPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
-
-        ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah");
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(2, flowFiles.size());
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4blahHello-5blahHello-6")).count());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-        assertEquals(2, mockPool.actualCommitOffsets.size());
-        assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-        assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset());
-    }
-
-    @Test
-    public void validatePollException() throws Exception {
-        String groupName = "validatePollException";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
-        mockPool.throwKafkaExceptionOnPoll = true;
-
-        ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo");
-        runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah");
-
-        runner.run(1, true);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(0, flowFiles.size());
-        assertNull(null, mockPool.actualCommitOffsets);
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertTrue(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-    }
-
-    @Test
-    public void validateCommitOffsetException() throws Exception {
-        String groupName = "validateCommitOffsetException";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
-        mockPool.throwKafkaExceptionOnCommit = true;
-
-        ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo");
-        runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR, "blah");
-
-        runner.run(1, true);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(1, flowFiles.size());
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertTrue(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-        assertNull(null, mockPool.actualCommitOffsets);
-    }
-
-    @Test
-    public void validateUtf8Key() {
-        String groupName = "validateGetAllMessages";
-
-        final Map<byte[], byte[]> rawRecords = new HashMap<>();
-        rawRecords.put("key1".getBytes(), "Hello-1".getBytes());
-        rawRecords.put(new byte[0], "Hello-2".getBytes());
-        rawRecords.put(null, "Hello-3".getBytes());
-
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, rawRecords);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.emptyMap(), null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-
-        ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "key1".equals(key)).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> key == null).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "".equals(key)).count());
-
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-    }
-
-    @Test
-    public void validateHexKey() {
-        String groupName = "validateGetAllMessages";
-
-        final Map<byte[], byte[]> rawRecords = new HashMap<>();
-        rawRecords.put("key1".getBytes(), "Hello-1".getBytes());
-        rawRecords.put(new byte[0], "Hello-2".getBytes());
-        rawRecords.put(null, "Hello-3".getBytes());
-
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, rawRecords);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.emptyMap(), null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-
-        ConsumeKafka_0_10 proc = new ConsumeKafka_0_10() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka_0_10.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka_0_10.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka_0_10.AUTO_OFFSET_RESET, ConsumeKafka_0_10.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka_0_10.KEY_ATTRIBUTE_ENCODING, ConsumeKafka_0_10.HEX_ENCODING);
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka_0_10.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        final String expectedHex = (Integer.toHexString('k') + Integer.toHexString('e') + Integer.toHexString('y') + Integer.toHexString('1')).toUpperCase();
-
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> expectedHex.equals(key)).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> key == null).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "".equals(key)).count());
-
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
index 7f88ea2..0ebf2b3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
@@ -16,109 +16,203 @@
  */
 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 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.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 static org.mockito.Matchers.anyInt;
+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 {
 
     Consumer<byte[], byte[]> consumer = null;
+    ProcessSession mockSession = null;
+    ProvenanceReporter mockReporter = null;
+    ConsumerPool testPool = null;
+    ConsumerPool testDemarcatedPool = null;
     ComponentLog logger = null;
 
     @Before
     public void setup() {
         consumer = mock(Consumer.class);
         logger = mock(ComponentLog.class);
-    }
-
-    @Test
-    public void validatePoolSimpleCreateClose() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
+        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) {
             @Override
             protected Consumer<byte[], byte[]> createKafkaConsumer() {
                 return consumer;
             }
         };
+        testDemarcatedPool = new ConsumerPool(
+                1,
+                "--demarcator--".getBytes(StandardCharsets.UTF_8),
+                Collections.emptyMap(),
+                Collections.singletonList("nifi"),
+                100L,
+                "utf-8",
+                "ssl",
+                "localhost",
+                logger) {
+            @Override
+            protected Consumer<byte[], byte[]> createKafkaConsumer() {
+                return consumer;
+            }
+        };
+    }
 
-        when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty());
+    @Test
+    public void validatePoolSimpleCreateClose() throws Exception {
 
-        try (final ConsumerLease lease = testPool.obtainConsumer()) {
+        when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
             lease.poll();
-            lease.commitOffsets(Collections.emptyMap());
         }
         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);
-        assertEquals(1, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
+        assertEquals(4, stats.leasesObtainedCount);
     }
 
     @Test
-    public void validatePoolSimpleBatchCreateClose() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(5, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
-            @Override
-            protected Consumer<byte[], byte[]> createKafkaConsumer() {
-                return consumer;
-            }
+    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<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
-        when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty());
+        when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            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()) {
+            try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
                 for (int j = 0; j < 100; j++) {
                     lease.poll();
                 }
-                lease.commitOffsets(Collections.emptyMap());
             }
         }
         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);
-        assertEquals(10000, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
     }
 
     @Test
-    public void validatePoolConsumerFails() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
-            @Override
-            protected Consumer<byte[], byte[]> createKafkaConsumer() {
-                return consumer;
-            }
+    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<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
-        when(consumer.poll(anyInt())).thenThrow(new KafkaException());
-
-        try (final ConsumerLease lease = testPool.obtainConsumer()) {
+        when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testDemarcatedPool.obtainConsumer(mockSession)) {
             lease.poll();
-            fail();
-        } catch (final KafkaException ke) {
+            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)) {
+            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);
-        assertEquals(0, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
     }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    static ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) {
+        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
+        final TopicPartition tPart = new TopicPartition(topic, partition);
+        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
+        long offset = startingOffset;
+        for (final byte[] rawRecord : rawRecords) {
+            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord);
+            records.add(rec);
+        }
+        map.put(tPart, records);
+        return new ConsumerRecords(map);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
deleted file mode 100644
index 19c64af..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
+++ /dev/null
@@ -1,306 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.kafka.clients.producer.Partitioner;
-import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult;
-import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
-import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.ConsumerTimeoutException;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import org.apache.kafka.clients.producer.ProducerConfig;
-
-public class KafkaPublisherTest {
-
-    private static EmbeddedKafka kafkaLocal;
-
-    private static EmbeddedKafkaProducerHelper producerHelper;
-
-    @BeforeClass
-    public static void beforeClass() {
-        kafkaLocal = new EmbeddedKafka();
-        kafkaLocal.start();
-        producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal);
-    }
-
-    @AfterClass
-    public static void afterClass() throws Exception {
-        producerHelper.close();
-        kafkaLocal.stop();
-    }
-
-    @Test
-    public void validateSuccessfulSendAsWhole() throws Exception {
-        InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateSuccessfulSendAsWhole";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        KafkaPublisherResult result = publisher.publish(publishingContext);
-
-        assertEquals(0, result.getLastMessageAcked());
-        assertEquals(1, result.getMessagesSent());
-        contentStream.close();
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        assertNotNull(iter.next());
-        try {
-            iter.next();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-    }
-
-    @Test
-    public void validateSuccessfulSendAsDelimited() throws Exception {
-        InputStream contentStream = new ByteArrayInputStream(
-                "Hello Kafka\nHello Kafka\nHello Kafka\nHello Kafka\n".getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateSuccessfulSendAsDelimited";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        KafkaPublisherResult result = publisher.publish(publishingContext);
-
-        assertEquals(3, result.getLastMessageAcked());
-        assertEquals(4, result.getMessagesSent());
-        contentStream.close();
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-    }
-
-    /*
-     * This test simulates the condition where not all messages were ACKed by
-     * Kafka
-     */
-    @Test
-    public void validateRetries() throws Exception {
-        byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
-        InputStream contentStream = new ByteArrayInputStream(testValue);
-        String topicName = "validateSuccessfulReSendOfFailedSegments";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        // simulates the first re-try
-        int lastAckedMessageIndex = 1;
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-
-        publisher.publish(publishingContext);
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        String m1 = new String(iter.next().message());
-        String m2 = new String(iter.next().message());
-        assertEquals("Hello Kafka3", m1);
-        assertEquals("Hello Kafka4", m2);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        // simulates the second re-try
-        lastAckedMessageIndex = 2;
-        contentStream = new ByteArrayInputStream(testValue);
-        publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        publisher.publish(publishingContext);
-
-        m1 = new String(iter.next().message());
-        assertEquals("Hello Kafka4", m1);
-
-        publisher.close();
-    }
-
-    /*
-     * Similar to the above test, but it sets the first retry index to the last
-     * possible message index and second index to an out of bound index. The
-     * expectation is that no messages will be sent to Kafka
-     */
-    @Test
-    public void validateRetriesWithWrongIndex() throws Exception {
-        byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
-        InputStream contentStream = new ByteArrayInputStream(testValue);
-        String topicName = "validateRetriesWithWrongIndex";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        // simulates the first re-try
-        int lastAckedMessageIndex = 3;
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-
-        publisher.publish(publishingContext);
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        // simulates the second re-try
-        lastAckedMessageIndex = 6;
-        contentStream = new ByteArrayInputStream(testValue);
-        publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        publisher.publish(publishingContext);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        publisher.close();
-    }
-
-    @Test
-    public void validateWithMultiByteCharactersNoDelimiter() throws Exception {
-        String data = "\u50e0THIS IS MY NEW TEXT.\u50e0IT HAS A NEWLINE.";
-        InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateWithMultiByteCharacters";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-
-        publisher.publish(publishingContext);
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        String r = new String(iter.next().message(), StandardCharsets.UTF_8);
-        assertEquals(data, r);
-    }
-
-    @Test
-    public void validateWithNonDefaultPartitioner() throws Exception {
-        String data = "fooandbarandbaz";
-        InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateWithNonDefaultPartitioner";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        kafkaProperties.setProperty("partitioner.class", TestPartitioner.class.getName());
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        publishingContext.setDelimiterBytes("and".getBytes(StandardCharsets.UTF_8));
-
-        try {
-            publisher.publish(publishingContext);
-            // partitioner should be invoked 3 times
-            assertTrue(TestPartitioner.counter == 3);
-            publisher.close();
-        } finally {
-            TestPartitioner.counter = 0;
-        }
-    }
-
-    private Properties buildProducerProperties() {
-        Properties kafkaProperties = new Properties();
-        kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort());
-        kafkaProperties.put("auto.create.topics.enable", "true");
-        return kafkaProperties;
-    }
-
-    private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
-        Properties props = new Properties();
-        props.put("zookeeper.connect", "localhost:" + kafkaLocal.getZookeeperPort());
-        props.put("group.id", "test");
-        props.put("consumer.timeout.ms", "500");
-        props.put("auto.offset.reset", "smallest");
-        ConsumerConfig consumerConfig = new ConsumerConfig(props);
-        ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig);
-        Map<String, Integer> topicCountMap = new HashMap<>(1);
-        topicCountMap.put(topic, 1);
-        Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
-        List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
-        ConsumerIterator<byte[], byte[]> iter = streams.get(0).iterator();
-        return iter;
-    }
-
-    public static class TestPartitioner implements Partitioner {
-
-        static int counter;
-
-        @Override
-        public void configure(Map<String, ?> configs) {
-            // nothing to do, test
-        }
-
-        @Override
-        public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes,
-                Cluster cluster) {
-            counter++;
-            return 0;
-        }
-
-        @Override
-        public void close() {
-            counter = 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
deleted file mode 100644
index af0d343..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
+++ /dev/null
@@ -1,375 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Test;
-import org.mockito.Mockito;
-import static org.mockito.Mockito.times;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.verify;
-
-public class PublishKafkaTest {
-
-    @Test
-    public void validateCustomSerilaizerDeserializerSettings() throws Exception {
-        PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
-        runner.setProperty(PublishKafka_0_10.TOPIC, "foo");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3 sec");
-        runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        runner.assertValid();
-        runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void validatePropertiesValidation() throws Exception {
-        PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
-        runner.setProperty(PublishKafka_0_10.TOPIC, "foo");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "foo");
-
-        try {
-            runner.assertValid();
-            fail();
-        } catch (AssertionError e) {
-            assertTrue(e.getMessage().contains("'max.block.ms' validated against 'foo' is invalid"));
-        }
-    }
-
-    @Test
-    public void validateCustomValidation() {
-        String topicName = "validateCustomValidation";
-        PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
-
-        /*
-         * Validates that Kerberos principle is required if one of SASL set for
-         * secirity protocol
-         */
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
-        try {
-            runner.run();
-            fail();
-        } catch (Throwable e) {
-            assertTrue(e.getMessage().contains("'Kerberos Service Name' is invalid because"));
-        }
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateSingleCharacterDemarcatedMessages() {
-        String topicName = "validateSingleCharacterDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-
-        runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerA() {
-        String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo");
-
-        runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerB() {
-        String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
-        StubPublishKafka putKafka = new StubPublishKafka(1);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo");
-
-        runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnSendFailureAndThenResendSuccessA() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3000 millis");
-
-        final String text = "Hello World\nGoodbye\nfail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-        putKafka.destroy();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnSendFailureAndThenResendSuccessB() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(1);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
-
-        final String text = "Hello World\nGoodbye\nfail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnFutureGetFailureAndThenResendSuccessFirstMessageFail() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
-
-        final String text = "futurefail\nHello World\nGoodbye\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0);
-        assertNotNull(ff);
-        runner.enqueue(ff);
-
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        // 6 sends due to duplication
-        verify(producer, times(5)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnFutureGetFailureAndThenResendSuccess() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
-
-        final String text = "Hello World\nGoodbye\nfuturefail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0);
-        assertNotNull(ff);
-        runner.enqueue(ff);
-
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        // 6 sends due to duplication
-        verify(producer, times(6)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateDemarcationIntoEmptyMessages() {
-        String topicName = "validateDemarcationIntoEmptyMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        final TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(PublishKafka_0_10.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
-
-        final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
-        runner.enqueue(bytes);
-        runner.run(1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexRightPartialDemarcatedMessages() {
-        String topicName = "validateComplexRightPartialDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0I Mean IT!\u50e0<\u50e0WILDSTUFF\u50e0>".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(3)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexLeftPartialDemarcatedMessages() {
-        String topicName = "validateComplexLeftPartialDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0I Mean IT!\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexPartialMatchDemarcatedMessages() {
-        String topicName = "validateComplexPartialMatchDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDBOOMSTUFF\u50e0>\u50e0".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(2)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @Test
-    public void validateUtf8Key() {
-        String topicName = "validateUtf8Key";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.KEY, "${myKey}");
-
-        final Map<String, String> attributes = Collections.singletonMap("myKey", "key1");
-        runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
-        runner.run(1);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
-        final Map<Object, Object> msgs = putKafka.getMessagesSent();
-        assertEquals(1, msgs.size());
-        final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
-        assertTrue(Arrays.equals("key1".getBytes(StandardCharsets.UTF_8), msgKey));
-    }
-
-    @Test
-    public void validateHexKey() {
-        String topicName = "validateUtf8Key";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka_0_10.KEY_ATTRIBUTE_ENCODING, PublishKafka_0_10.HEX_ENCODING);
-        runner.setProperty(PublishKafka_0_10.KEY, "${myKey}");
-
-        final Map<String, String> attributes = Collections.singletonMap("myKey", "6B657931");
-        runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
-        runner.run(1);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
-        final Map<Object, Object> msgs = putKafka.getMessagesSent();
-        assertEquals(1, msgs.size());
-        final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
-
-        assertTrue(Arrays.equals(new byte[] {0x6B, 0x65, 0x79, 0x31}, msgKey));
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
deleted file mode 100644
index 76c29cd..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-
-import org.junit.Test;
-
-public class PublishingContextTest {
-
-    @Test
-    public void failInvalidConstructorArgs() {
-        try {
-            new PublishingContext(null, null);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-        try {
-            new PublishingContext(mock(InputStream.class), null);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        try {
-            new PublishingContext(mock(InputStream.class), "");
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        try {
-            new PublishingContext(mock(InputStream.class), "mytopic", -3);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-    }
-
-    @Test
-    public void validateFullSetting() {
-        PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic", 3);
-        publishingContext.setDelimiterBytes("delimiter".getBytes(StandardCharsets.UTF_8));
-        publishingContext.setKeyBytes("key".getBytes(StandardCharsets.UTF_8));
-
-        assertEquals("delimiter", new String(publishingContext.getDelimiterBytes(), StandardCharsets.UTF_8));
-        assertEquals("key", new String(publishingContext.getKeyBytes(), StandardCharsets.UTF_8));
-        assertEquals("topic", publishingContext.getTopic());
-        assertEquals("topic: 'topic'; delimiter: 'delimiter'", publishingContext.toString());
-    }
-
-    @Test
-    public void validateOnlyOnceSetPerInstance() {
-        PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic");
-        publishingContext.setKeyBytes(new byte[]{0});
-        try {
-            publishingContext.setKeyBytes(new byte[]{0});
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        publishingContext.setDelimiterBytes(new byte[]{0});
-        try {
-            publishingContext.setDelimiterBytes(new byte[]{0});
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
deleted file mode 100644
index c009014..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.lang.reflect.Field;
-import static org.mockito.Mockito.when;
-
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.TopicAuthorizationException;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.exception.ProcessException;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS;
-import org.mockito.Mockito;
-import static org.mockito.Mockito.mock;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class StubPublishKafka extends PublishKafka_0_10 {
-
-    private volatile Producer<byte[], byte[]> producer;
-
-    private volatile boolean failed;
-
-    private final int ackCheckSize;
-
-    private final ExecutorService executor = Executors.newCachedThreadPool();
-    private final Map<Object, Object> msgsSent = new ConcurrentHashMap<>();
-
-    StubPublishKafka(int ackCheckSize) {
-        this.ackCheckSize = ackCheckSize;
-    }
-
-    public Producer<byte[], byte[]> getProducer() {
-        return producer;
-    }
-
-    public void destroy() {
-        this.executor.shutdownNow();
-    }
-
-    public Map<Object, Object> getMessagesSent() {
-        return new HashMap<>(msgsSent);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
-            throws ProcessException {
-        final Map<String, String> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
-        kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        KafkaPublisher publisher;
-        try {
-            Field f = PublishKafka_0_10.class.getDeclaredField("brokers");
-            f.setAccessible(true);
-            f.set(this, context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue());
-            publisher = (KafkaPublisher) TestUtils.getUnsafe().allocateInstance(KafkaPublisher.class);
-            publisher.setAckWaitTime(15000);
-            producer = mock(Producer.class);
-            this.instrumentProducer(producer, false);
-            Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer");
-            kf.setAccessible(true);
-            kf.set(publisher, producer);
-
-            Field componentLogF = KafkaPublisher.class.getDeclaredField("componentLog");
-            componentLogF.setAccessible(true);
-            componentLogF.set(publisher, mock(ComponentLog.class));
-
-            Field ackCheckSizeField = KafkaPublisher.class.getDeclaredField("ackCheckSize");
-            ackCheckSizeField.setAccessible(true);
-            ackCheckSizeField.set(publisher, this.ackCheckSize);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IllegalStateException(e);
-        }
-        return publisher;
-    }
-
-    @SuppressWarnings("unchecked")
-    private void instrumentProducer(Producer<byte[], byte[]> producer, boolean failRandomly) {
-
-        when(producer.send(Mockito.any(ProducerRecord.class))).then(new Answer<Future<RecordMetadata>>() {
-            @Override
-            public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
-                final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
-                if (record != null && record.key() != null) {
-                    msgsSent.put(record.key(), record.value());
-                }
-
-                String value = new String(record.value(), StandardCharsets.UTF_8);
-                if ("fail".equals(value) && !StubPublishKafka.this.failed) {
-                    StubPublishKafka.this.failed = true;
-                    throw new RuntimeException("intentional");
-                }
-                Future<RecordMetadata> future = executor.submit(new Callable<RecordMetadata>() {
-                    @Override
-                    public RecordMetadata call() throws Exception {
-                        if ("futurefail".equals(value) && !StubPublishKafka.this.failed) {
-                            StubPublishKafka.this.failed = true;
-                            throw new TopicAuthorizationException("Unauthorized");
-                        } else {
-                            TopicPartition partition = new TopicPartition("foo", 0);
-                            RecordMetadata meta = new RecordMetadata(partition, 0, 0);
-                            return meta;
-                        }
-                    }
-                });
-                return future;
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
new file mode 100644
index 0000000..e54a10c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
@@ -0,0 +1,87 @@
+/*
+ * 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.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();
+        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();
+        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
+        }
+    }
+
+}


[07/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
index 7e4b12c..8e3fa3b 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java
@@ -16,105 +16,36 @@
  */
 package org.apache.nifi.processors.kafka.pubsub;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.UUID;
-
+import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
+
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+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;
 
 public class ConsumeKafkaTest {
 
-    static class MockConsumerPool extends ConsumerPool {
-
-        final int actualMaxLeases;
-        final List<String> actualTopics;
-        final Map<String, String> actualKafkaProperties;
-        boolean throwKafkaExceptionOnPoll = false;
-        boolean throwKafkaExceptionOnCommit = false;
-        Queue<ConsumerRecords<byte[], byte[]>> nextPlannedRecordsQueue = new ArrayDeque<>();
-        Map<TopicPartition, OffsetAndMetadata> nextExpectedCommitOffsets = null;
-        Map<TopicPartition, OffsetAndMetadata> actualCommitOffsets = null;
-        boolean wasConsumerLeasePoisoned = false;
-        boolean wasConsumerLeaseClosed = false;
-        boolean wasPoolClosed = false;
-
-        public MockConsumerPool(int maxLeases, List<String> topics, Map<String, String> kafkaProperties, ComponentLog logger) {
-            super(maxLeases, topics, kafkaProperties, null);
-            actualMaxLeases = maxLeases;
-            actualTopics = topics;
-            actualKafkaProperties = kafkaProperties;
-        }
-
-        @Override
-        public ConsumerLease obtainConsumer() {
-            return new ConsumerLease() {
-                @Override
-                public ConsumerRecords<byte[], byte[]> poll() {
-                    if (throwKafkaExceptionOnPoll) {
-                        throw new KafkaException("i planned to fail");
-                    }
-                    final ConsumerRecords<byte[], byte[]> records = nextPlannedRecordsQueue.poll();
-                    return (records == null) ? ConsumerRecords.empty() : records;
-                }
-
-                @Override
-                public void commitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) {
-                    if (throwKafkaExceptionOnCommit) {
-                        throw new KafkaException("i planned to fail");
-                    }
-                    actualCommitOffsets = offsets;
-                }
-
-                @Override
-                public void poison() {
-                    wasConsumerLeasePoisoned = true;
-                }
-
-                @Override
-                public void close() {
-                    wasConsumerLeaseClosed = true;
-                }
-            };
-        }
-
-        @Override
-        public void close() {
-            wasPoolClosed = true;
-        }
-
-        void resetState() {
-            throwKafkaExceptionOnPoll = false;
-            throwKafkaExceptionOnCommit = false;
-            nextPlannedRecordsQueue = null;
-            nextExpectedCommitOffsets = null;
-            wasConsumerLeasePoisoned = false;
-            wasConsumerLeaseClosed = false;
-            wasPoolClosed = false;
-        }
+    Consumer<byte[], byte[]> mockConsumer = null;
+    ConsumerLease mockLease = null;
+    ConsumerPool mockConsumerPool = null;
 
+    @Before
+    public void setup() {
+        mockConsumer = mock(Consumer.class);
+        mockLease = mock(ConsumerLease.class);
+        mockConsumerPool = mock(ConsumerPool.class);
     }
 
     @Test
@@ -175,365 +106,45 @@ public class ConsumeKafkaTest {
     public void validateGetAllMessages() throws Exception {
         String groupName = "validateGetAllMessages";
 
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
-
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-        mockPool.nextPlannedRecordsQueue.add(secondRecs);
-
-        ConsumeKafka proc = new ConsumeKafka() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        if (mockPool.nextPlannedRecordsQueue.isEmpty()) {
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4")).count());
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-5")).count());
-            assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-6")).count());
-            assertEquals(2, mockPool.actualCommitOffsets.size());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset());
-        } else {
-            assertEquals(2, mockPool.actualCommitOffsets.size());
-            assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-        }
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-    }
-
-    @Test
-    public void validateGetLotsOfMessages() throws Exception {
-        String groupName = "validateGetLotsOfMessages";
-
-        final byte[][] firstPassValues = new byte[10010][1];
-        for (final byte[] value : firstPassValues) {
-            value[0] = 0x12;
-        }
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
-
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> secondRecs = createConsumerRecords("bar", 1, 1L, secondPassValues);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-        mockPool.nextPlannedRecordsQueue.add(secondRecs);
-
-        ConsumeKafka proc = new ConsumeKafka() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(10010, flowFiles.stream().map(ff -> ff.toByteArray()).filter(content -> content.length == 1 && content[0] == 0x12).count());
-        assertEquals(1, mockPool.nextPlannedRecordsQueue.size());
-
-        assertEquals(1, mockPool.actualCommitOffsets.size());
-        assertEquals(10011L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-    }
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        final TopicPartition tPart = new TopicPartition(topic, partition);
-        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
-        long offset = startingOffset;
-        for (final byte[] rawRecord : rawRecords) {
-            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord);
-            records.add(rec);
-        }
-        map.put(tPart, records);
-        return new ConsumerRecords(map);
-    }
-
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    private ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final Map<byte[], byte[]> rawRecords) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        final TopicPartition tPart = new TopicPartition(topic, partition);
-        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
-        long offset = startingOffset;
-
-        for (final Map.Entry<byte[], byte[]> entry : rawRecords.entrySet()) {
-            final byte[] key = entry.getKey();
-            final byte[] rawRecord = entry.getValue();
-            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, key, rawRecord);
-            records.add(rec);
-        }
-        map.put(tPart, records);
-        return new ConsumerRecords(map);
-    }
-
-    private ConsumerRecords<byte[], byte[]> mergeRecords(final ConsumerRecords<byte[], byte[]>... records) {
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
-        for (final ConsumerRecords<byte[], byte[]> rec : records) {
-            rec.partitions().stream().forEach((part) -> {
-                final List<ConsumerRecord<byte[], byte[]>> conRecs = rec.records(part);
-                if (map.get(part) != null) {
-                    throw new IllegalStateException("already have that topic/partition in the record map");
-                }
-                map.put(part, conRecs);
-            });
-        }
-        return new ConsumerRecords<>(map);
-    }
-
-    @Test
-    public void validateGetAllMessagesWithProvidedDemarcator() throws Exception {
-        String groupName = "validateGetAllMessagesWithProvidedDemarcator";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final byte[][] secondPassValues = new byte[][]{
-            "Hello-4".getBytes(StandardCharsets.UTF_8),
-            "Hello-5".getBytes(StandardCharsets.UTF_8),
-            "Hello-6".getBytes(StandardCharsets.UTF_8)
-        };
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues),
-                createConsumerRecords("bar", 1, 1L, secondPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
+        when(mockConsumerPool.obtainConsumer(anyObject())).thenReturn(mockLease);
+        when(mockLease.continuePolling()).thenReturn(Boolean.TRUE, Boolean.TRUE, Boolean.FALSE);
+        when(mockLease.commit()).thenReturn(Boolean.TRUE);
 
         ConsumeKafka proc = new ConsumeKafka() {
             @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
+            protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
+                return mockConsumerPool;
             }
         };
-
         final TestRunner runner = TestRunners.newTestRunner(proc);
         runner.setValidateExpressionUsage(false);
         runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
         runner.setProperty(ConsumeKafka.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah");
-
         runner.run(1, false);
 
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(2, flowFiles.size());
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-4blahHello-5blahHello-6")).count());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-        assertEquals(2, mockPool.actualCommitOffsets.size());
-        assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("foo", 1)).offset());
-        assertEquals(4L, mockPool.actualCommitOffsets.get(new TopicPartition("bar", 1)).offset());
-    }
-
-    @Test
-    public void validatePollException() throws Exception {
-        String groupName = "validatePollException";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
-        mockPool.throwKafkaExceptionOnPoll = true;
-
-        ConsumeKafka proc = new ConsumeKafka() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka.TOPICS, "foo");
-        runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah");
-
-        runner.run(1, true);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(0, flowFiles.size());
-        assertNull(null, mockPool.actualCommitOffsets);
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertTrue(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-    }
-
-    @Test
-    public void validateCommitOffsetException() throws Exception {
-        String groupName = "validateCommitOffsetException";
-
-        final byte[][] firstPassValues = new byte[][]{
-            "Hello-1".getBytes(StandardCharsets.UTF_8),
-            "Hello-2".getBytes(StandardCharsets.UTF_8),
-            "Hello-3".getBytes(StandardCharsets.UTF_8)
-        };
-
-        final ConsumerRecords<byte[], byte[]> consumerRecs = mergeRecords(
-                createConsumerRecords("foo", 1, 1L, firstPassValues)
-        );
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.EMPTY_MAP, null);
-        mockPool.nextPlannedRecordsQueue.add(consumerRecs);
-        mockPool.throwKafkaExceptionOnCommit = true;
-
-        ConsumeKafka proc = new ConsumeKafka() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka.TOPICS, "foo");
-        runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah");
-
-        runner.run(1, true);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(1, flowFiles.size());
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1blahHello-2blahHello-3")).count());
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertTrue(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-
-        assertNull(null, mockPool.actualCommitOffsets);
+        verify(mockConsumerPool, times(1)).obtainConsumer(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 validateUtf8Key() {
-        String groupName = "validateGetAllMessages";
-
-        final Map<byte[], byte[]> rawRecords = new HashMap<>();
-        rawRecords.put("key1".getBytes(), "Hello-1".getBytes());
-        rawRecords.put(new byte[0], "Hello-2".getBytes());
-        rawRecords.put(null, "Hello-3".getBytes());
+    public void validateGetErrorMessages() throws Exception {
+        String groupName = "validateGetErrorMessages";
 
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, rawRecords);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.emptyMap(), null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
+        when(mockConsumerPool.obtainConsumer(anyObject())).thenReturn(mockLease);
+        when(mockLease.continuePolling()).thenReturn(true, false);
+        when(mockLease.commit()).thenReturn(Boolean.FALSE);
 
         ConsumeKafka proc = new ConsumeKafka() {
             @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
+            protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
+                return mockConsumerPool;
             }
         };
         final TestRunner runner = TestRunners.newTestRunner(proc);
@@ -542,89 +153,15 @@ public class ConsumeKafkaTest {
         runner.setProperty(ConsumeKafka.TOPICS, "foo,bar");
         runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
         runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-
         runner.run(1, false);
 
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "key1".equals(key)).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> key == null).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "".equals(key)).count());
-
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
+        verify(mockConsumerPool, times(1)).obtainConsumer(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 validateHexKey() {
-        String groupName = "validateGetAllMessages";
-
-        final Map<byte[], byte[]> rawRecords = new HashMap<>();
-        rawRecords.put("key1".getBytes(), "Hello-1".getBytes());
-        rawRecords.put(new byte[0], "Hello-2".getBytes());
-        rawRecords.put(null, "Hello-3".getBytes());
-
-        final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, rawRecords);
-
-        final List<String> expectedTopics = new ArrayList<>();
-        expectedTopics.add("foo");
-        expectedTopics.add("bar");
-        final MockConsumerPool mockPool = new MockConsumerPool(1, expectedTopics, Collections.emptyMap(), null);
-        mockPool.nextPlannedRecordsQueue.add(firstRecs);
-
-        ConsumeKafka proc = new ConsumeKafka() {
-            @Override
-            protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-                return mockPool;
-            }
-        };
-        final TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
-        runner.setProperty(ConsumeKafka.TOPICS, "foo,bar");
-        runner.setProperty(ConsumeKafka.GROUP_ID, groupName);
-        runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
-        runner.setProperty(ConsumeKafka.KEY_ATTRIBUTE_ENCODING, ConsumeKafka.HEX_ENCODING);
-
-        runner.run(1, false);
-
-        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
-
-        assertEquals(expectedTopics, mockPool.actualTopics);
-
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-1")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-2")).count());
-        assertEquals(1, flowFiles.stream().map(ff -> new String(ff.toByteArray())).filter(content -> content.equals("Hello-3")).count());
-
-        final String expectedHex = (Integer.toHexString('k') + Integer.toHexString('e') + Integer.toHexString('y') + Integer.toHexString('1')).toUpperCase();
-
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> expectedHex.equals(key)).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> key == null).count());
-        assertEquals(1, flowFiles.stream().map(ff -> ff.getAttribute(KafkaProcessorUtils.KAFKA_KEY)).filter(key -> "".equals(key)).count());
-
-
-        //asert that all consumers were closed as expected
-        //assert that the consumer pool was properly closed
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertFalse(mockPool.wasPoolClosed);
-        runner.run(1, true);
-        assertFalse(mockPool.wasConsumerLeasePoisoned);
-        assertTrue(mockPool.wasConsumerLeaseClosed);
-        assertTrue(mockPool.wasPoolClosed);
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
index 7f88ea2..0ebf2b3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
@@ -16,109 +16,203 @@
  */
 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 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.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 static org.mockito.Matchers.anyInt;
+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 {
 
     Consumer<byte[], byte[]> consumer = null;
+    ProcessSession mockSession = null;
+    ProvenanceReporter mockReporter = null;
+    ConsumerPool testPool = null;
+    ConsumerPool testDemarcatedPool = null;
     ComponentLog logger = null;
 
     @Before
     public void setup() {
         consumer = mock(Consumer.class);
         logger = mock(ComponentLog.class);
-    }
-
-    @Test
-    public void validatePoolSimpleCreateClose() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
+        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) {
             @Override
             protected Consumer<byte[], byte[]> createKafkaConsumer() {
                 return consumer;
             }
         };
+        testDemarcatedPool = new ConsumerPool(
+                1,
+                "--demarcator--".getBytes(StandardCharsets.UTF_8),
+                Collections.emptyMap(),
+                Collections.singletonList("nifi"),
+                100L,
+                "utf-8",
+                "ssl",
+                "localhost",
+                logger) {
+            @Override
+            protected Consumer<byte[], byte[]> createKafkaConsumer() {
+                return consumer;
+            }
+        };
+    }
 
-        when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty());
+    @Test
+    public void validatePoolSimpleCreateClose() throws Exception {
 
-        try (final ConsumerLease lease = testPool.obtainConsumer()) {
+        when(consumer.poll(anyLong())).thenReturn(createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            lease.poll();
+        }
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
             lease.poll();
-            lease.commitOffsets(Collections.emptyMap());
         }
         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);
-        assertEquals(1, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
+        assertEquals(4, stats.leasesObtainedCount);
     }
 
     @Test
-    public void validatePoolSimpleBatchCreateClose() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(5, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
-            @Override
-            protected Consumer<byte[], byte[]> createKafkaConsumer() {
-                return consumer;
-            }
+    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<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
-        when(consumer.poll(anyInt())).thenReturn(ConsumerRecords.empty());
+        when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
+            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()) {
+            try (final ConsumerLease lease = testPool.obtainConsumer(mockSession)) {
                 for (int j = 0; j < 100; j++) {
                     lease.poll();
                 }
-                lease.commitOffsets(Collections.emptyMap());
             }
         }
         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);
-        assertEquals(10000, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
     }
 
     @Test
-    public void validatePoolConsumerFails() throws Exception {
-
-        final ConsumerPool testPool = new ConsumerPool(1, Collections.singletonList("nifi"), Collections.emptyMap(), logger) {
-            @Override
-            protected Consumer<byte[], byte[]> createKafkaConsumer() {
-                return consumer;
-            }
+    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<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
-        when(consumer.poll(anyInt())).thenThrow(new KafkaException());
-
-        try (final ConsumerLease lease = testPool.obtainConsumer()) {
+        when(consumer.poll(anyLong())).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        try (final ConsumerLease lease = testDemarcatedPool.obtainConsumer(mockSession)) {
             lease.poll();
-            fail();
-        } catch (final KafkaException ke) {
+            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)) {
+            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);
-        assertEquals(0, stats.unproductivePollCount);
-        assertEquals(0, stats.productivePollCount);
     }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    static ConsumerRecords<byte[], byte[]> createConsumerRecords(final String topic, final int partition, final long startingOffset, final byte[][] rawRecords) {
+        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> map = new HashMap<>();
+        final TopicPartition tPart = new TopicPartition(topic, partition);
+        final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
+        long offset = startingOffset;
+        for (final byte[] rawRecord : rawRecords) {
+            final ConsumerRecord<byte[], byte[]> rec = new ConsumerRecord(topic, partition, offset++, UUID.randomUUID().toString().getBytes(), rawRecord);
+            records.add(rec);
+        }
+        map.put(tPart, records);
+        return new ConsumerRecords(map);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
deleted file mode 100644
index 19c64af..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisherTest.java
+++ /dev/null
@@ -1,306 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.kafka.clients.producer.Partitioner;
-import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult;
-import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
-import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.ConsumerTimeoutException;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import org.apache.kafka.clients.producer.ProducerConfig;
-
-public class KafkaPublisherTest {
-
-    private static EmbeddedKafka kafkaLocal;
-
-    private static EmbeddedKafkaProducerHelper producerHelper;
-
-    @BeforeClass
-    public static void beforeClass() {
-        kafkaLocal = new EmbeddedKafka();
-        kafkaLocal.start();
-        producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal);
-    }
-
-    @AfterClass
-    public static void afterClass() throws Exception {
-        producerHelper.close();
-        kafkaLocal.stop();
-    }
-
-    @Test
-    public void validateSuccessfulSendAsWhole() throws Exception {
-        InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateSuccessfulSendAsWhole";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        KafkaPublisherResult result = publisher.publish(publishingContext);
-
-        assertEquals(0, result.getLastMessageAcked());
-        assertEquals(1, result.getMessagesSent());
-        contentStream.close();
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        assertNotNull(iter.next());
-        try {
-            iter.next();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-    }
-
-    @Test
-    public void validateSuccessfulSendAsDelimited() throws Exception {
-        InputStream contentStream = new ByteArrayInputStream(
-                "Hello Kafka\nHello Kafka\nHello Kafka\nHello Kafka\n".getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateSuccessfulSendAsDelimited";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        KafkaPublisherResult result = publisher.publish(publishingContext);
-
-        assertEquals(3, result.getLastMessageAcked());
-        assertEquals(4, result.getMessagesSent());
-        contentStream.close();
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        assertNotNull(iter.next());
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-    }
-
-    /*
-     * This test simulates the condition where not all messages were ACKed by
-     * Kafka
-     */
-    @Test
-    public void validateRetries() throws Exception {
-        byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
-        InputStream contentStream = new ByteArrayInputStream(testValue);
-        String topicName = "validateSuccessfulReSendOfFailedSegments";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        // simulates the first re-try
-        int lastAckedMessageIndex = 1;
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-
-        publisher.publish(publishingContext);
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        String m1 = new String(iter.next().message());
-        String m2 = new String(iter.next().message());
-        assertEquals("Hello Kafka3", m1);
-        assertEquals("Hello Kafka4", m2);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        // simulates the second re-try
-        lastAckedMessageIndex = 2;
-        contentStream = new ByteArrayInputStream(testValue);
-        publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        publisher.publish(publishingContext);
-
-        m1 = new String(iter.next().message());
-        assertEquals("Hello Kafka4", m1);
-
-        publisher.close();
-    }
-
-    /*
-     * Similar to the above test, but it sets the first retry index to the last
-     * possible message index and second index to an out of bound index. The
-     * expectation is that no messages will be sent to Kafka
-     */
-    @Test
-    public void validateRetriesWithWrongIndex() throws Exception {
-        byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
-        InputStream contentStream = new ByteArrayInputStream(testValue);
-        String topicName = "validateRetriesWithWrongIndex";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-
-        // simulates the first re-try
-        int lastAckedMessageIndex = 3;
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-
-        publisher.publish(publishingContext);
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        // simulates the second re-try
-        lastAckedMessageIndex = 6;
-        contentStream = new ByteArrayInputStream(testValue);
-        publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
-        publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
-        publisher.publish(publishingContext);
-        try {
-            iter.next();
-            fail();
-        } catch (ConsumerTimeoutException e) {
-            // that's OK since this is the Kafka mechanism to unblock
-        }
-
-        publisher.close();
-    }
-
-    @Test
-    public void validateWithMultiByteCharactersNoDelimiter() throws Exception {
-        String data = "\u50e0THIS IS MY NEW TEXT.\u50e0IT HAS A NEWLINE.";
-        InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateWithMultiByteCharacters";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-
-        publisher.publish(publishingContext);
-        publisher.close();
-
-        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
-        String r = new String(iter.next().message(), StandardCharsets.UTF_8);
-        assertEquals(data, r);
-    }
-
-    @Test
-    public void validateWithNonDefaultPartitioner() throws Exception {
-        String data = "fooandbarandbaz";
-        InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
-        String topicName = "validateWithNonDefaultPartitioner";
-
-        Properties kafkaProperties = this.buildProducerProperties();
-        kafkaProperties.setProperty("partitioner.class", TestPartitioner.class.getName());
-        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
-        publishingContext.setDelimiterBytes("and".getBytes(StandardCharsets.UTF_8));
-
-        try {
-            publisher.publish(publishingContext);
-            // partitioner should be invoked 3 times
-            assertTrue(TestPartitioner.counter == 3);
-            publisher.close();
-        } finally {
-            TestPartitioner.counter = 0;
-        }
-    }
-
-    private Properties buildProducerProperties() {
-        Properties kafkaProperties = new Properties();
-        kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort());
-        kafkaProperties.put("auto.create.topics.enable", "true");
-        return kafkaProperties;
-    }
-
-    private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
-        Properties props = new Properties();
-        props.put("zookeeper.connect", "localhost:" + kafkaLocal.getZookeeperPort());
-        props.put("group.id", "test");
-        props.put("consumer.timeout.ms", "500");
-        props.put("auto.offset.reset", "smallest");
-        ConsumerConfig consumerConfig = new ConsumerConfig(props);
-        ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig);
-        Map<String, Integer> topicCountMap = new HashMap<>(1);
-        topicCountMap.put(topic, 1);
-        Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
-        List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
-        ConsumerIterator<byte[], byte[]> iter = streams.get(0).iterator();
-        return iter;
-    }
-
-    public static class TestPartitioner implements Partitioner {
-
-        static int counter;
-
-        @Override
-        public void configure(Map<String, ?> configs) {
-            // nothing to do, test
-        }
-
-        @Override
-        public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes,
-                Cluster cluster) {
-            counter++;
-            return 0;
-        }
-
-        @Override
-        public void close() {
-            counter = 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
deleted file mode 100644
index d81f0c1..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaTest.java
+++ /dev/null
@@ -1,375 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Test;
-import org.mockito.Mockito;
-import static org.mockito.Mockito.times;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.verify;
-
-public class PublishKafkaTest {
-
-    @Test
-    public void validateCustomSerilaizerDeserializerSettings() throws Exception {
-        PublishKafka publishKafka = new PublishKafka();
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
-        runner.setProperty(PublishKafka.TOPIC, "foo");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "3 sec");
-        runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        runner.assertValid();
-        runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void validatePropertiesValidation() throws Exception {
-        PublishKafka publishKafka = new PublishKafka();
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
-        runner.setProperty(PublishKafka.TOPIC, "foo");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "foo");
-
-        try {
-            runner.assertValid();
-            fail();
-        } catch (AssertionError e) {
-            assertTrue(e.getMessage().contains("'max.block.ms' validated against 'foo' is invalid"));
-        }
-    }
-
-    @Test
-    public void validateCustomValidation() {
-        String topicName = "validateCustomValidation";
-        PublishKafka publishKafka = new PublishKafka();
-
-        /*
-         * Validates that Kerberos principle is required if one of SASL set for
-         * secirity protocol
-         */
-        TestRunner runner = TestRunners.newTestRunner(publishKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
-        try {
-            runner.run();
-            fail();
-        } catch (Throwable e) {
-            assertTrue(e.getMessage().contains("'Kerberos Service Name' is invalid because"));
-        }
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateSingleCharacterDemarcatedMessages() {
-        String topicName = "validateSingleCharacterDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-
-        runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerA() {
-        String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo");
-
-        runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerB() {
-        String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
-        StubPublishKafka putKafka = new StubPublishKafka(1);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo");
-
-        runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
-
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnSendFailureAndThenResendSuccessA() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "3000 millis");
-
-        final String text = "Hello World\nGoodbye\nfail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-        putKafka.destroy();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnSendFailureAndThenResendSuccessB() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(1);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
-
-        final String text = "Hello World\nGoodbye\nfail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnFutureGetFailureAndThenResendSuccessFirstMessageFail() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
-
-        final String text = "futurefail\nHello World\nGoodbye\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka.REL_FAILURE).get(0);
-        assertNotNull(ff);
-        runner.enqueue(ff);
-
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        // 6 sends due to duplication
-        verify(producer, times(5)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateOnFutureGetFailureAndThenResendSuccess() throws Exception {
-        String topicName = "validateSendFailureAndThenResendSuccess";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-        runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
-
-        final String text = "Hello World\nGoodbye\nfuturefail\n2";
-        runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-        MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka.REL_FAILURE).get(0);
-        assertNotNull(ff);
-        runner.enqueue(ff);
-
-        runner.run(1, false);
-        assertEquals(0, runner.getQueueSize().getObjectCount());
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        // 6 sends due to duplication
-        verify(producer, times(6)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateDemarcationIntoEmptyMessages() {
-        String topicName = "validateDemarcationIntoEmptyMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        final TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(PublishKafka.KEY, "key1");
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
-
-        final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
-        runner.enqueue(bytes);
-        runner.run(1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexRightPartialDemarcatedMessages() {
-        String topicName = "validateComplexRightPartialDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0I Mean IT!\u50e0<\u50e0WILDSTUFF\u50e0>".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(3)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexLeftPartialDemarcatedMessages() {
-        String topicName = "validateComplexLeftPartialDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0I Mean IT!\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void validateComplexPartialMatchDemarcatedMessages() {
-        String topicName = "validateComplexPartialMatchDemarcatedMessages";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0");
-
-        runner.enqueue("Hello World\u50e0<\u50e0WILDSTUFF\u50e0>\u50e0Goodbye\u50e0<\u50e0WILDBOOMSTUFF\u50e0>\u50e0".getBytes(StandardCharsets.UTF_8));
-        runner.run(1, false);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
-        Producer<byte[], byte[]> producer = putKafka.getProducer();
-        verify(producer, times(2)).send(Mockito.any(ProducerRecord.class));
-        runner.shutdown();
-    }
-
-    @Test
-    public void validateUtf8Key() {
-        String topicName = "validateUtf8Key";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.KEY, "${myKey}");
-
-        final Map<String, String> attributes = Collections.singletonMap("myKey", "key1");
-        runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
-        runner.run(1);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
-        final Map<Object, Object> msgs = putKafka.getMessagesSent();
-        assertEquals(1, msgs.size());
-        final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
-        assertTrue(Arrays.equals("key1".getBytes(StandardCharsets.UTF_8), msgKey));
-    }
-
-    @Test
-    public void validateHexKey() {
-        String topicName = "validateUtf8Key";
-        StubPublishKafka putKafka = new StubPublishKafka(100);
-        TestRunner runner = TestRunners.newTestRunner(putKafka);
-        runner.setProperty(PublishKafka.TOPIC, topicName);
-        runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
-        runner.setProperty(PublishKafka.KEY_ATTRIBUTE_ENCODING, PublishKafka.HEX_ENCODING);
-        runner.setProperty(PublishKafka.KEY, "${myKey}");
-
-        final Map<String, String> attributes = Collections.singletonMap("myKey", "6B657931");
-        runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
-        runner.run(1);
-
-        runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
-        final Map<Object, Object> msgs = putKafka.getMessagesSent();
-        assertEquals(1, msgs.size());
-        final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
-
-        assertTrue(Arrays.equals(new byte[] {0x6B, 0x65, 0x79, 0x31}, msgKey));
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
deleted file mode 100644
index 76c29cd..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/PublishingContextTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-
-import org.junit.Test;
-
-public class PublishingContextTest {
-
-    @Test
-    public void failInvalidConstructorArgs() {
-        try {
-            new PublishingContext(null, null);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-        try {
-            new PublishingContext(mock(InputStream.class), null);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        try {
-            new PublishingContext(mock(InputStream.class), "");
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        try {
-            new PublishingContext(mock(InputStream.class), "mytopic", -3);
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-    }
-
-    @Test
-    public void validateFullSetting() {
-        PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic", 3);
-        publishingContext.setDelimiterBytes("delimiter".getBytes(StandardCharsets.UTF_8));
-        publishingContext.setKeyBytes("key".getBytes(StandardCharsets.UTF_8));
-
-        assertEquals("delimiter", new String(publishingContext.getDelimiterBytes(), StandardCharsets.UTF_8));
-        assertEquals("key", new String(publishingContext.getKeyBytes(), StandardCharsets.UTF_8));
-        assertEquals("topic", publishingContext.getTopic());
-        assertEquals("topic: 'topic'; delimiter: 'delimiter'", publishingContext.toString());
-    }
-
-    @Test
-    public void validateOnlyOnceSetPerInstance() {
-        PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic");
-        publishingContext.setKeyBytes(new byte[]{0});
-        try {
-            publishingContext.setKeyBytes(new byte[]{0});
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-
-        publishingContext.setDelimiterBytes(new byte[]{0});
-        try {
-            publishingContext.setDelimiterBytes(new byte[]{0});
-            fail();
-        } catch (IllegalArgumentException e) {
-            // success
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
deleted file mode 100644
index 533655e..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/StubPublishKafka.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.lang.reflect.Field;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.TopicAuthorizationException;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class StubPublishKafka extends PublishKafka {
-
-    private volatile Producer<byte[], byte[]> producer;
-
-    private volatile boolean failed;
-
-    private final int ackCheckSize;
-
-    private final ExecutorService executor = Executors.newCachedThreadPool();
-    private final Map<Object, Object> msgsSent = new ConcurrentHashMap<>();
-
-    StubPublishKafka(int ackCheckSize) {
-        this.ackCheckSize = ackCheckSize;
-    }
-
-    public Producer<byte[], byte[]> getProducer() {
-        return producer;
-    }
-
-    public void destroy() {
-        this.executor.shutdownNow();
-    }
-
-    public Map<Object, Object> getMessagesSent() {
-        return new HashMap<>(msgsSent);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
-            throws ProcessException {
-        final Map<String, String> kafkaProperties = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
-        kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        KafkaPublisher publisher;
-        try {
-            Field f = PublishKafka.class.getDeclaredField("brokers");
-            f.setAccessible(true);
-            f.set(this, context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue());
-            publisher = (KafkaPublisher) TestUtils.getUnsafe().allocateInstance(KafkaPublisher.class);
-            publisher.setAckWaitTime(15000);
-            producer = mock(Producer.class);
-
-            this.instrumentProducer(producer, false);
-            Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer");
-            kf.setAccessible(true);
-            kf.set(publisher, producer);
-
-            Field componentLogF = KafkaPublisher.class.getDeclaredField("componentLog");
-            componentLogF.setAccessible(true);
-            componentLogF.set(publisher, mock(ComponentLog.class));
-
-            Field ackCheckSizeField = KafkaPublisher.class.getDeclaredField("ackCheckSize");
-            ackCheckSizeField.setAccessible(true);
-            ackCheckSizeField.set(publisher, this.ackCheckSize);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new IllegalStateException(e);
-        }
-        return publisher;
-    }
-
-    @SuppressWarnings("unchecked")
-    private void instrumentProducer(Producer<byte[], byte[]> producer, boolean failRandomly) {
-
-        when(producer.send(Mockito.any(ProducerRecord.class))).then(new Answer<Future<RecordMetadata>>() {
-            @Override
-            public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
-                final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
-                if (record != null && record.key() != null) {
-                    msgsSent.put(record.key(), record.value());
-                }
-
-                String value = new String(record.value(), StandardCharsets.UTF_8);
-                if ("fail".equals(value) && !StubPublishKafka.this.failed) {
-                    StubPublishKafka.this.failed = true;
-                    throw new RuntimeException("intentional");
-                }
-                Future<RecordMetadata> future = executor.submit(new Callable<RecordMetadata>() {
-                    @Override
-                    public RecordMetadata call() throws Exception {
-                        if ("futurefail".equals(value) && !StubPublishKafka.this.failed) {
-                            StubPublishKafka.this.failed = true;
-                            throw new TopicAuthorizationException("Unauthorized");
-                        } else {
-                            TopicPartition partition = new TopicPartition("foo", 0);
-                            RecordMetadata meta = new RecordMetadata(partition, 0, 0);
-                            return meta;
-                        }
-                    }
-                });
-                return future;
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
new file mode 100644
index 0000000..e54a10c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestInFlightMessageTracker.java
@@ -0,0 +1,87 @@
+/*
+ * 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.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();
+        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();
+        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
+        }
+    }
+
+}


[02/20] nifi git commit: NIFI-2694: - Addressing instances where phase one objects were being referenced instead of phase two objects. - Code clean up.

Posted by jp...@apache.org.
NIFI-2694: - Addressing instances where phase one objects were being referenced instead of phase two objects. - Code clean up.

This closes #970.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 94bc7c656f896d306d907da2d33dc5d9b1ac9426
Parents: d5f94c4
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Aug 30 10:07:09 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:20:26 2016 -0500

----------------------------------------------------------------------
 .../apache/nifi/web/api/ControllerResource.java |  5 +--
 .../apache/nifi/web/api/CountersResource.java   |  5 ++-
 .../org/apache/nifi/web/api/LabelResource.java  |  2 +-
 .../nifi/web/api/ProcessGroupResource.java      | 32 +++++++++++---------
 4 files changed, 25 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/94bc7c65/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 1fe87ab..09abe53 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -677,9 +677,9 @@ public class ControllerResource extends ApplicationResource {
                     authorizeController(RequestAction.WRITE);
                 },
                 null,
-                (endDateEtity) -> {
+                (endDateEntity) -> {
                     // purge the actions
-                    serviceFacade.deleteActions(endDateEtity.getEndDate());
+                    serviceFacade.deleteActions(endDateEntity.getEndDate());
 
                     // generate the response
                     return generateOkResponse(new HistoryEntity()).build();
@@ -700,6 +700,7 @@ public class ControllerResource extends ApplicationResource {
     }
 
     // setters
+
     public void setServiceFacade(final NiFiServiceFacade serviceFacade) {
         this.serviceFacade = serviceFacade;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/94bc7c65/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/CountersResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/CountersResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/CountersResource.java
index a8be20d..e3a2b9e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/CountersResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/CountersResource.java
@@ -226,6 +226,9 @@ public class CountersResource extends ApplicationResource {
     )
     public Response updateCounter(
             @Context final HttpServletRequest httpServletRequest,
+            @ApiParam(
+                    value = "The id of the counter."
+            )
             @PathParam("id") final String id) {
 
         if (isReplicateRequest()) {
@@ -244,7 +247,7 @@ public class CountersResource extends ApplicationResource {
                 null,
                 (componentEntity) -> {
                     // reset the specified counter
-                    final CounterDTO counter = serviceFacade.updateCounter(requestComponentEntity.getId());
+                    final CounterDTO counter = serviceFacade.updateCounter(componentEntity.getId());
 
                     // create the response entity
                     final CounterEntity entity = new CounterEntity();

http://git-wip-us.apache.org/repos/asf/nifi/blob/94bc7c65/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java
index 64ddde3..78c7287 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java
@@ -173,7 +173,7 @@ public class LabelResource extends ApplicationResource {
             )
             @PathParam("id") final String id,
             @ApiParam(
-                    value = "The label configuraiton details.",
+                    value = "The label configuration details.",
                     required = true
             ) final LabelEntity requestLabelEntity) {
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/94bc7c65/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index b348828..527364c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -387,10 +387,10 @@ public class ProcessGroupResource extends ApplicationResource {
                     // delete the process group
                     final ProcessGroupEntity entity = serviceFacade.deleteProcessGroup(revision, processGroupEntity.getId());
 
-                        // create the response
-                        return clusterContext(generateOkResponse(entity)).build();
-                    }
-            );
+                    // create the response
+                    return clusterContext(generateOkResponse(entity)).build();
+                }
+        );
     }
 
     /**
@@ -548,7 +548,7 @@ public class ProcessGroupResource extends ApplicationResource {
      *
      * @param httpServletRequest request
      * @param groupId            The group id
-     * @param processorEntity    A processorEntity.
+     * @param requestProcessorEntity    A processorEntity.
      * @return A processorEntity.
      */
     @POST
@@ -582,17 +582,17 @@ public class ProcessGroupResource extends ApplicationResource {
             @ApiParam(
                     value = "The processor configuration details.",
                     required = true
-            ) final ProcessorEntity processorEntity) {
+            ) final ProcessorEntity requestProcessorEntity) {
 
-        if (processorEntity == null || processorEntity.getComponent() == null) {
+        if (requestProcessorEntity == null || requestProcessorEntity.getComponent() == null) {
             throw new IllegalArgumentException("Processor details must be specified.");
         }
 
-        if (processorEntity.getRevision() == null || (processorEntity.getRevision().getVersion() == null || processorEntity.getRevision().getVersion() != 0)) {
+        if (requestProcessorEntity.getRevision() == null || (requestProcessorEntity.getRevision().getVersion() == null || requestProcessorEntity.getRevision().getVersion() != 0)) {
             throw new IllegalArgumentException("A revision of 0 must be specified when creating a new Processor.");
         }
 
-        final ProcessorDTO requestProcessor = processorEntity.getComponent();
+        final ProcessorDTO requestProcessor = requestProcessorEntity.getComponent();
         if (requestProcessor.getId() != null) {
             throw new IllegalArgumentException("Processor ID cannot be specified.");
         }
@@ -608,12 +608,12 @@ public class ProcessGroupResource extends ApplicationResource {
         requestProcessor.setParentGroupId(groupId);
 
         if (isReplicateRequest()) {
-            return replicate(HttpMethod.POST, processorEntity);
+            return replicate(HttpMethod.POST, requestProcessorEntity);
         }
 
         return withWriteLock(
                 serviceFacade,
-                processorEntity,
+                requestProcessorEntity,
                 lookup -> {
                     final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
                     processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
@@ -625,13 +625,15 @@ public class ProcessGroupResource extends ApplicationResource {
                     }
                 },
                 null,
-                procEntity -> {
+                processorEntity -> {
+                    final ProcessorDTO processor = processorEntity.getComponent();
+
                     // set the processor id as appropriate
-                    requestProcessor.setId(generateUuid());
+                    processor.setId(generateUuid());
 
                     // create the new processor
-                    final Revision revision = getRevision(processorEntity, requestProcessor.getId());
-                    final ProcessorEntity entity = serviceFacade.createProcessor(revision, groupId, requestProcessor);
+                    final Revision revision = getRevision(processorEntity, processor.getId());
+                    final ProcessorEntity entity = serviceFacade.createProcessor(revision, groupId, processor);
                     processorResource.populateRemainingProcessorEntityContent(entity);
 
                     // generate a 201 created response


[04/20] nifi git commit: NIFI-2797: - Correcting download URIs for OTPs.

Posted by jp...@apache.org.
NIFI-2797: - Correcting download URIs for OTPs.

This closes #1038.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 3feb59d690ad1d2f301c8572d09b262299bd23a5
Parents: bde1c24
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Sep 20 12:50:55 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:20:39 2016 -0500

----------------------------------------------------------------------
 .../nifi/web/security/otp/OtpAuthenticationFilter.java    |  6 +++---
 .../web/security/otp/OtpAuthenticationFilterTest.java     | 10 +++++-----
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/3feb59d6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilter.java
index 1ab0b41..34883fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilter.java
@@ -31,11 +31,11 @@ public class OtpAuthenticationFilter extends NiFiAuthenticationFilter {
     private static final Logger logger = LoggerFactory.getLogger(OtpAuthenticationFilter.class);
 
     private static final Pattern PROVENANCE_DOWNLOAD_PATTERN =
-        Pattern.compile("/controller/provenance/events/[0-9]+/content/(?:(?:output)|(?:input))");
+        Pattern.compile("/provenance-events/([0-9]+)/content/((?:input)|(?:output))");
     private static final Pattern QUEUE_DOWNLOAD_PATTERN =
-        Pattern.compile("/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/connections/[a-f0-9\\-]{36}/flowfiles/[a-f0-9\\-]{36}/content");
+        Pattern.compile("/flowfile-queues/([a-f0-9\\-]{36})/flowfiles/([a-f0-9\\-]{36})/content");
     private static final Pattern TEMPLATE_DOWNLOAD_PATTERN =
-        Pattern.compile("/controller/templates/[a-f0-9\\-]{36}");
+        Pattern.compile("/templates/[a-f0-9\\-]{36}/download");
 
     protected static final String ACCESS_TOKEN = "access_token";
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/3feb59d6/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
index 791ca54..8027b8f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/otp/OtpAuthenticationFilterTest.java
@@ -65,7 +65,7 @@ public class OtpAuthenticationFilterTest {
         when(request.isSecure()).thenReturn(true);
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn("my-access-token");
         when(request.getContextPath()).thenReturn("/nifi-api");
-        when(request.getPathInfo()).thenReturn("/controller/config");
+        when(request.getPathInfo()).thenReturn("/flow/cluster/summary");
 
         assertNull(otpAuthenticationFilter.attemptAuthentication(request));
     }
@@ -88,7 +88,7 @@ public class OtpAuthenticationFilterTest {
         when(request.isSecure()).thenReturn(true);
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn(DOWNLOAD_TOKEN);
         when(request.getContextPath()).thenReturn("/nifi-api");
-        when(request.getPathInfo()).thenReturn("/controller/provenance/events/0/content/input");
+        when(request.getPathInfo()).thenReturn("/provenance-events/0/content/input");
 
         final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
         assertEquals(DOWNLOAD_TOKEN, result.getToken());
@@ -101,7 +101,7 @@ public class OtpAuthenticationFilterTest {
         when(request.isSecure()).thenReturn(true);
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn(DOWNLOAD_TOKEN);
         when(request.getContextPath()).thenReturn("/nifi-api");
-        when(request.getPathInfo()).thenReturn("/controller/provenance/events/0/content/output");
+        when(request.getPathInfo()).thenReturn("/provenance-events/0/content/output");
 
         final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
         assertEquals(DOWNLOAD_TOKEN, result.getToken());
@@ -116,7 +116,7 @@ public class OtpAuthenticationFilterTest {
         when(request.isSecure()).thenReturn(true);
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn(DOWNLOAD_TOKEN);
         when(request.getContextPath()).thenReturn("/nifi-api");
-        when(request.getPathInfo()).thenReturn(String.format("/controller/process-groups/root/connections/%s/flowfiles/%s/content", uuid, uuid));
+        when(request.getPathInfo()).thenReturn(String.format("/flowfile-queues/%s/flowfiles/%s/content", uuid, uuid));
 
         final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
         assertEquals(DOWNLOAD_TOKEN, result.getToken());
@@ -131,7 +131,7 @@ public class OtpAuthenticationFilterTest {
         when(request.isSecure()).thenReturn(true);
         when(request.getParameter(OtpAuthenticationFilter.ACCESS_TOKEN)).thenReturn(DOWNLOAD_TOKEN);
         when(request.getContextPath()).thenReturn("/nifi-api");
-        when(request.getPathInfo()).thenReturn(String.format("/controller/templates/%s", uuid));
+        when(request.getPathInfo()).thenReturn(String.format("/templates/%s/download", uuid));
 
         final OtpAuthenticationRequestToken result = (OtpAuthenticationRequestToken) otpAuthenticationFilter.attemptAuthentication(request);
         assertEquals(DOWNLOAD_TOKEN, result.getToken());


[05/20] nifi git commit: NIFI-2824: - Updating replication logic to account for the potential replication target and then invoking the corresponding action.

Posted by jp...@apache.org.
NIFI-2824: - Updating replication logic to account for the potential replication target and then invoking the corresponding action.

Signed-off-by: Yolanda M. Davis <ym...@apache.org>

This closes #1068


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 1ba7f8302c2d6573ff9f43acd49e644213c480f6
Parents: 3feb59d
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Sep 26 16:03:07 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:20:44 2016 -0500

----------------------------------------------------------------------
 .../web/StandardNiFiWebConfigurationContext.java   | 17 ++++++++++++-----
 1 file changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/1ba7f830/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index 021f216..fb38ce9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -49,6 +49,7 @@ import org.apache.nifi.controller.reporting.ReportingTaskProvider;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.api.ApplicationResource.ReplicationTarget;
 import org.apache.nifi.web.api.dto.AllowableValueDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
@@ -72,14 +73,12 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URLEncoder;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
 
 /**
  * Implements the NiFiWebConfigurationContext interface to support a context in both standalone and clustered environments.
@@ -292,17 +291,25 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
         return componentFacade.updateComponent(requestContext, annotationData, properties);
     }
 
+    private ReplicationTarget getReplicationTarget() {
+        return clusterCoordinator.isActiveClusterCoordinator() ? ReplicationTarget.CLUSTER_NODES : ReplicationTarget.CLUSTER_COORDINATOR;
+    }
+
     private NodeResponse replicate(final String method, final URI uri, final Object entity, final Map<String, String> headers) throws InterruptedException {
         final NodeIdentifier coordinatorNode = clusterCoordinator.getElectedActiveCoordinatorNode();
         if (coordinatorNode == null) {
             throw new NoClusterCoordinatorException();
         }
 
-        final Set<NodeIdentifier> coordinatorNodes = Collections.singleton(coordinatorNode);
-        return requestReplicator.replicate(coordinatorNodes, method, uri, entity, headers, false, true).awaitMergedResponse();
+        // Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly
+        // to the cluster nodes themselves.
+        if (getReplicationTarget() == ReplicationTarget.CLUSTER_NODES) {
+            return requestReplicator.replicate(method, uri, entity, headers).awaitMergedResponse();
+        } else {
+            return requestReplicator.forwardToCoordinator(coordinatorNode, method, uri, entity, headers).awaitMergedResponse();
+        }
     }
 
-
     /**
      * Facade over accessing different types of NiFi components.
      */


[20/20] nifi git commit: NIFI-2717: Ensure that when performing site-to-site communications that we always commit the transaction on the remote side before committing it on the local side

Posted by jp...@apache.org.
NIFI-2717: Ensure that when performing site-to-site communications that we always commit the transaction on the remote side before committing it on the local side


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 5d81cabdc7817dbf6f9901ffa6a1981ce3e57123
Parents: e80a670
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Aug 31 15:02:30 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Thu Dec 15 12:06:32 2016 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5d81cabd/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index 1996357..e8b542e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -305,8 +305,8 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
             final String dataSize = FormatUtils.formatDataSize(bytesSent);
 
-            session.commit();
             transaction.complete();
+            session.commit();
 
             final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
             logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[]{


[13/20] nifi git commit: [NIFI-3154] display ellipsis when text is sufficiently long in connection details and connection configuration dialogs. This closes #1305

Posted by jp...@apache.org.
[NIFI-3154] display ellipsis when text is sufficiently long in connection details and connection configuration dialogs. This closes #1305


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 7c213deb6ba001169c27b97a4f28608851582902
Parents: ed17df5
Author: Scott Aslan <sc...@gmail.com>
Authored: Thu Dec 8 10:46:26 2016 -0500
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:23:55 2016 -0500

----------------------------------------------------------------------
 .../canvas/connection-configuration.jsp         | 18 ++++-----
 .../WEB-INF/partials/connection-details.jsp     |  4 +-
 .../src/main/webapp/css/common-ui.css           |  1 +
 .../webapp/css/connection-configuration.css     |  2 -
 .../src/main/webapp/css/connection-details.css  |  2 -
 .../main/webapp/css/processor-configuration.css |  1 +
 .../js/nf/canvas/nf-connection-configuration.js | 14 +++----
 .../main/webapp/js/nf/nf-connection-details.js  | 40 ++++++++++----------
 8 files changed, 40 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/connection-configuration.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/connection-configuration.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/connection-configuration.jsp
index a93bea0..9e65743 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/connection-configuration.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/connection-configuration.jsp
@@ -95,7 +95,7 @@
                     <div id="read-only-output-port-source" class="setting hidden">
                         <div class="setting-name">From output</div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="read-only-output-port-name"></div>
+                            <div id="read-only-output-port-name" class="ellipsis"></div>
                         </div>
                     </div>
                     <div id="output-port-source" class="setting hidden">
@@ -107,21 +107,21 @@
                     <div id="input-port-source" class="setting hidden">
                         <div class="setting-name">From input</div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="input-port-source-name" class="label"></div>
+                            <div id="input-port-source-name" class="label ellipsis"></div>
                         </div>
                     </div>
                     <div id="funnel-source" class="setting hidden">
                         <div class="setting-name">From funnel</div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="funnel-source-name" class="label">funnel</div>
+                            <div id="funnel-source-name" class="label ellipsis" title="funnel">funnel</div>
                         </div>
                     </div>
                     <div id="processor-source" class="setting hidden">
                         <div class="setting-name">From processor</div>
                         <div class="setting-field connection-terminal-label">
                             <div id="processor-source-details">
-                                <div id="processor-source-name" class="label"></div>
-                                <div id="processor-source-type"></div>
+                                <div id="processor-source-name" class="label ellipsis"></div>
+                                <div id="processor-source-type" class="ellipsis"></div>
                             </div>
                         </div>
                     </div>
@@ -149,21 +149,21 @@
                     <div id="output-port-destination" class="setting hidden">
                         <div class="setting-name">To output</div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="output-port-destination-name" class="label"></div>
+                            <div id="output-port-destination-name" class="label ellipsis"></div>
                         </div>
                     </div>
                     <div id="funnel-destination" class="setting hidden">
                         <div class="setting-name">To funnel</div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="funnel-source-name" class="label">funnel</div>
+                            <div id="funnel-destination-name" class="label ellipsis" title="funnel">funnel</div>
                         </div>
                     </div>
                     <div id="processor-destination" class="setting hidden">
                         <div class="setting-name">To processor</div>
                         <div class="setting-field connection-terminal-label">
                             <div id="processor-destination-details">
-                                <div id="processor-destination-name" class="label"></div>
-                                <div id="processor-destination-type"></div>
+                                <div id="processor-destination-name" class="label ellipsis"></div>
+                                <div id="processor-destination-type" class="ellipsis"></div>
                             </div>
                         </div>
                     </div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/connection-details.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/connection-details.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/connection-details.jsp
index 2bbd480..f7637a2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/connection-details.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/connection-details.jsp
@@ -24,7 +24,7 @@
                     <div class="setting">
                         <div id="read-only-connection-source-label" class="setting-name"></div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="read-only-connection-source"></div>
+                            <div id="read-only-connection-source" class="ellipsis"></div>
                         </div>
                     </div>
                     <div class="setting">
@@ -48,7 +48,7 @@
                     <div class="setting">
                         <div id="read-only-connection-target-label" class="setting-name"></div>
                         <div class="setting-field connection-terminal-label">
-                            <div id="read-only-connection-target"></div>
+                            <div id="read-only-connection-target" class="ellipsis"></div>
                         </div>
                     </div>
                     <div class="setting">

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
index 9eac226..7dec556 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
@@ -162,6 +162,7 @@ ul.links span.header-link-over {
 .ellipsis {
     white-space: nowrap;
     overflow: hidden;
+    text-overflow: ellipsis;
 }
 
 .ellipsis.multiline {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-configuration.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-configuration.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-configuration.css
index 30487f9..b18da8d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-configuration.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-configuration.css
@@ -46,7 +46,6 @@ div.connection-terminal-label {
 #relationship-names {
     padding: 2px;
     height: 150px;
-    width: 274px;
     overflow-y: auto;
     overflow-x: hidden; 
     border: 0 solid #cccccc;
@@ -61,7 +60,6 @@ div.available-relationship-container {
 #connection-configuration div.relationship-name {
     display: inline-block;
     line-height: normal;
-    width: 240px;
 }
 
 #connection-name-container {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-details.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-details.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-details.css
index 61d9109..ce94485 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-details.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/connection-details.css
@@ -35,11 +35,9 @@
     overflow-x: hidden;
     overflow-y: auto;
     padding: 2px 2px 0;
-    width: 274px;
 }
 
 #connection-details div.relationship-name {
     display: inline-block;
     line-height: normal;
-    width: 240px;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/processor-configuration.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/processor-configuration.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/processor-configuration.css
index e03a698..0115287 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/processor-configuration.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/processor-configuration.css
@@ -110,6 +110,7 @@ div.relationship-description {
 #run-duration-slider {
     margin-top: 5px;
     border-radius: 0;
+    margin-right: 2px;
 }
 
 #run-duration-slider .ui-slider-handle {

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection-configuration.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection-configuration.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection-configuration.js
index b9672a2..74fc03d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection-configuration.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-connection-configuration.js
@@ -189,7 +189,7 @@ nf.ConnectionConfiguration = (function () {
 
             // populate the port information
             $('#input-port-source').show();
-            $('#input-port-source-name').text(inputPortName);
+            $('#input-port-source-name').text(inputPortName).attr('title', inputPortName);
 
             // populate the connection source details
             $('#connection-source-id').val(inputPortData.id);
@@ -244,8 +244,8 @@ nf.ConnectionConfiguration = (function () {
 
             // populate the source processor information
             $('#processor-source').show();
-            $('#processor-source-name').text(processorName);
-            $('#processor-source-type').text(processorType);
+            $('#processor-source-name').text(processorName).attr('title', processorName);
+            $('#processor-source-type').text(processorType).attr('title', processorType);
 
             // populate the connection source details
             $('#connection-source-id').val(processorData.id);
@@ -443,7 +443,7 @@ nf.ConnectionConfiguration = (function () {
             var outputPortName = outputPortData.permissions.canRead ? outputPortData.component.name : outputPortData.id;
 
             $('#output-port-destination').show();
-            $('#output-port-destination-name').text(outputPortName);
+            $('#output-port-destination-name').text(outputPortName).attr('title', outputPortName);
 
             // populate the connection destination details
             $('#connection-destination-id').val(outputPortData.id);
@@ -482,8 +482,8 @@ nf.ConnectionConfiguration = (function () {
             var processorType = processorData.permissions.canRead ? nf.Common.substringAfterLast(processorData.component.type, '.') : 'Processor';
 
             $('#processor-destination').show();
-            $('#processor-destination-name').text(processorName);
-            $('#processor-destination-type').text(processorType);
+            $('#processor-destination-name').text(processorName).attr('title', processorName);
+            $('#processor-destination-type').text(processorType).attr('title', processorType);
 
             // populate the connection destination details
             $('#connection-destination-id').val(processorData.id);
@@ -1307,7 +1307,7 @@ nf.ConnectionConfiguration = (function () {
                     if (nf.CanvasUtils.isProcessGroup(source) || nf.CanvasUtils.isRemoteProcessGroup(source)) {
                         // populate the connection source details
                         $('#connection-source-id').val(connection.source.id);
-                        $('#read-only-output-port-name').text(connection.source.name);
+                        $('#read-only-output-port-name').text(connection.source.name).attr('title', connection.source.name);
                     }
 
                     // if the destination is a process gorup or remote process group, select the appropriate port if applicable

http://git-wip-us.apache.org/repos/asf/nifi/blob/7c213deb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-connection-details.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-connection-details.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-connection-details.js
index 7c671ab..caed999 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-connection-details.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-connection-details.js
@@ -53,8 +53,8 @@ nf.ConnectionDetails = (function () {
                 dataType: 'json'
             }).done(function (response) {
                 var processor = response.component;
-                var processorName = $('<div class="label"></div>').text(processor.name);
-                var processorType = $('<div></div>').text(nf.Common.substringAfterLast(processor.type, '.'));
+                var processorName = $('<div class="label"></div>').text(processor.name).addClass('ellipsis').attr('title', processor.name);
+                var processorType = $('<div></div>').text(nf.Common.substringAfterLast(processor.type, '.')).addClass('ellipsis').attr('title', nf.Common.substringAfterLast(processor.type, '.'));
 
                 // populate source processor details
                 $('#read-only-connection-source-label').text('From processor');
@@ -64,8 +64,8 @@ nf.ConnectionDetails = (function () {
                 deferred.resolve();
             }).fail(function (xhr, status, error) {
                 if (xhr.status === 403) {
-                    var processorName = $('<div class="label"></div>').text(source.name);
-                    var processorType = $('<div></div>').text('Processor');
+                    var processorName = $('<div class="label"></div>').text(source.name).addClass('ellipsis').attr('title', source.name);
+                    var processorType = $('<div></div>').text('Processor').addClass('ellipsis').attr('title', 'Processor');
 
                     // populate source processor details
                     $('#read-only-connection-source-label').text('From processor');
@@ -90,7 +90,7 @@ nf.ConnectionDetails = (function () {
     var initializeSourceFunnel = function (groupId, groupName, source) {
         return $.Deferred(function (deferred) {
             $('#read-only-connection-source-label').text('From funnel');
-            $('#read-only-connection-source').append('funnel');
+            $('#read-only-connection-source').text('funnel').attr('title', 'funnel');
             $('#read-only-connection-source-group-name').text(groupName);
             deferred.resolve();
         }).promise();
@@ -114,13 +114,13 @@ nf.ConnectionDetails = (function () {
 
                 // populate source port details
                 $('#read-only-connection-source-label').text('From output');
-                $('#read-only-connection-source').text(source.name);
+                $('#read-only-connection-source').text(source.name).attr('title', source.name);
                 $('#read-only-connection-source-group-name').text(remoteProcessGroup.name);
             }).fail(function (xhr, status, error) {
                 if (xhr.status === 403) {
                     // populate source processor details
                     $('#read-only-connection-source-label').text('From output');
-                    $('#read-only-connection-source').append(source.name);
+                    $('#read-only-connection-source').text(source.name).attr('title', source.name);
                     $('#read-only-connection-source-group-name').text(source.groupId);
 
                     deferred.resolve();
@@ -143,7 +143,7 @@ nf.ConnectionDetails = (function () {
             if (groupId === source.groupId) {
                 // populate source port details
                 $('#read-only-connection-source-label').text('From input');
-                $('#read-only-connection-source').text(source.name);
+                $('#read-only-connection-source').text(source.name).attr('title', source.name);
                 $('#read-only-connection-source-group-name').text(groupName);
 
                 deferred.resolve();
@@ -157,7 +157,7 @@ nf.ConnectionDetails = (function () {
 
                     // populate source port details
                     $('#read-only-connection-source-label').text('From output');
-                    $('#read-only-connection-source').text(source.name);
+                    $('#read-only-connection-source').text(source.name).attr('title', source.name);
                     $('#read-only-connection-source-group-name').text(processGroup.name);
 
                     deferred.resolve();
@@ -165,7 +165,7 @@ nf.ConnectionDetails = (function () {
                     if (xhr.status === 403) {
                         // populate source processor details
                         $('#read-only-connection-source-label').text('From output');
-                        $('#read-only-connection-source').append(source.name);
+                        $('#read-only-connection-source').text(source.name).attr('title', source.name);
                         $('#read-only-connection-source-group-name').text(source.groupId);
 
                         deferred.resolve();
@@ -211,8 +211,8 @@ nf.ConnectionDetails = (function () {
                 dataType: 'json'
             }).done(function (response) {
                 var processor = response.component;
-                var processorName = $('<div class="label"></div>').text(processor.name);
-                var processorType = $('<div></div>').text(nf.Common.substringAfterLast(processor.type, '.'));
+                var processorName = $('<div class="label"></div>').text(processor.name).addClass('ellipsis').attr('title', processor.name);
+                var processorType = $('<div></div>').text(nf.Common.substringAfterLast(processor.type, '.')).addClass('ellipsis').attr('title', nf.Common.substringAfterLast(processor.type, '.'));
 
                 // populate destination processor details
                 $('#read-only-connection-target-label').text('To processor');
@@ -222,8 +222,8 @@ nf.ConnectionDetails = (function () {
                 deferred.resolve();
             }).fail(function (xhr, status, error) {
                 if (xhr.status === 403) {
-                    var processorName = $('<div class="label"></div>').text(destination.name);
-                    var processorType = $('<div></div>').text('Processor');
+                    var processorName = $('<div class="label"></div>').text(destination.name).addClass('ellipsis').attr('title', destination.name);
+                    var processorType = $('<div></div>').text('Processor').addClass('ellipsis').attr('title', 'Processor');
 
                     // populate destination processor details
                     $('#read-only-connection-target-label').text('To processor');
@@ -248,7 +248,7 @@ nf.ConnectionDetails = (function () {
     var initializeDestinationFunnel = function (groupId, groupName, destination) {
         return $.Deferred(function (deferred) {
             $('#read-only-connection-target-label').text('To funnel');
-            $('#read-only-connection-target').append('funnel');
+            $('#read-only-connection-target').text('funnel').attr('title', 'funnel');
             $('#read-only-connection-target-group-name').text(groupName);
             deferred.resolve();
         }).promise();
@@ -272,7 +272,7 @@ nf.ConnectionDetails = (function () {
 
                 // populate source port details
                 $('#read-only-connection-target-label').text('To input');
-                $('#read-only-connection-target').text(destination.name);
+                $('#read-only-connection-target').text(destination.name).attr('title', destination.name);
                 $('#read-only-connection-target-group-name').text(remoteProcessGroup.name);
 
                 deferred.resolve();
@@ -280,7 +280,7 @@ nf.ConnectionDetails = (function () {
                 if (xhr.status === 403) {
                     // populate source port details
                     $('#read-only-connection-target-label').text('To input');
-                    $('#read-only-connection-target').append(destination.name);
+                    $('#read-only-connection-target').text(destination.name).attr('title', destination.name);
                     $('#read-only-connection-target-group-name').text(destination.groupId);
 
                     deferred.resolve();
@@ -303,7 +303,7 @@ nf.ConnectionDetails = (function () {
             if (groupId === destination.groupId) {
                 // populate destination port details
                 $('#read-only-connection-target-label').text('To output');
-                $('#read-only-connection-target').text(destination.name);
+                $('#read-only-connection-target').text(destination.name).attr('title', destination.name);
                 $('#read-only-connection-target-group-name').text(groupName);
 
                 deferred.resolve();
@@ -317,7 +317,7 @@ nf.ConnectionDetails = (function () {
 
                     // populate destination port details
                     $('#read-only-connection-target-label').text('To input');
-                    $('#read-only-connection-target').text(destination.name);
+                    $('#read-only-connection-target').text(destination.name).attr('title', destination.name);
                     $('#read-only-connection-target-group-name').text(processGroup.name);
 
                     deferred.resolve();
@@ -325,7 +325,7 @@ nf.ConnectionDetails = (function () {
                     if (xhr.status === 403) {
                         // populate source port details
                         $('#read-only-connection-target-label').text('To input');
-                        $('#read-only-connection-target').append(destination.name);
+                        $('#read-only-connection-target').text(destination.name).attr('title', destination.name);
                         $('#read-only-connection-target-group-name').text(destination.groupId);
 
                         deferred.resolve();


[14/20] nifi git commit: NIFI-3173: When a template is created with multiple components in different groups all referencing the same controller service, ensure that controller service is added to the template at a high enough level that all components ne

Posted by jp...@apache.org.
NIFI-3173: When a template is created with multiple components in different groups all referencing the same controller service, ensure that controller service is added to the template at a high enough level that all components needing hte service can access it.

- Ensure that controller services are added to child process groups when creating snippet

- Addressed issue related to modifying higher-level process groups' controller services in snippet after having already visited the process group

This closes #1318

Signed-off-by: jpercivall <JP...@apache.org>


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

Branch: refs/heads/support/nifi-1.0.x
Commit: 47b368f59b6f4a75c8e1712aa3fbd7263170c2a0
Parents: 7c213de
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Dec 11 20:36:28 2016 -0500
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:24:58 2016 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/web/util/SnippetUtils.java  | 96 +++++++++++++++++---
 1 file changed, 85 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/47b368f5/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
index c5baa8c..83a2861 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
@@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -109,10 +110,48 @@ public final class SnippetUtils {
             throw new IllegalStateException("The parent process group for this snippet could not be found.");
         }
 
-        final Set<ControllerServiceDTO> controllerServices = new HashSet<>();
+        // We need to ensure that the Controller Services that are added get added to the proper group.
+        // This can potentially get a little bit tricky. Consider this scenario:
+        // We have a Process Group G1. Within Process Group G1 is a Controller Service C1.
+        // Also within G1 is a child Process Group, G2. Within G2 is a child Process Group, G3.
+        // Within G3 are two child Process Groups: G4 and G5. Within each of these children,
+        // we have a Processor (P1, P2) that references the Controller Service C1, defined 3 levels above.
+        // Now, we create a template that encompasses only Process Groups G4 and G5. We need to ensure
+        // that the Controller Service C1 is included at the 'root' of the template so that those
+        // Processors within G4 and G5 both have access to the same Controller Service. This can be drawn
+        // out thus:
+        //
+        // G1 -- C1
+        // |
+        // |
+        // G2
+        // |
+        // |
+        // G3
+        // |  \
+        // |   \
+        // G4   G5
+        // |    |
+        // |    |
+        // P1   P2
+        //
+        // Both P1 and P2 reference C1.
+        //
+        // In order to accomplish this, we maintain two collections. First, we keep a Set of all Controller Services that have
+        // been added. If we add a new Controller Service to the set, then we know it hasn't been added anywhere in the Snippet.
+        // In that case, we determine the service's group ID. In the flow described above, if we template just groups G4 and G5,
+        // then we need to include the Controller Service defined at G1. So we also keep a Map of Group ID to controller services
+        // in that group. If the ParentGroupId of a Controller Service is not in our snippet, then we instead update the parent
+        // ParentGroupId to be that of our highest-level process group (in this case G3, as that's where the template is created)
+        // and then add the controller services to that group (NOTE: here, when we say we change the group ID and add to that group,
+        // we are talking only about the DTO objects that make up the snippet. We do not actually modify the Process Group or the
+        // Controller Services in our flow themselves!)
         final Set<ControllerServiceDTO> allServicesReferenced = new HashSet<>();
+        final Map<String, FlowSnippetDTO> contentsByGroup = new HashMap<>();
+        contentsByGroup.put(processGroup.getIdentifier(), snippetDto);
 
         // add any processors
+        final Set<ControllerServiceDTO> controllerServices = new HashSet<>();
         final Set<ProcessorDTO> processors = new LinkedHashSet<>();
         if (!snippet.getProcessors().isEmpty()) {
             for (final String processorId : snippet.getProcessors().keySet()) {
@@ -126,7 +165,12 @@ public final class SnippetUtils {
                     // Include all referenced services that are not already included in this snippet.
                     getControllerServices(processor.getProperties()).stream()
                         .filter(svc -> allServicesReferenced.add(svc))
-                        .forEach(svc -> controllerServices.add(svc));
+                        .forEach(svc -> {
+                            final String svcGroupId = svc.getParentGroupId();
+                            final String destinationGroupId = contentsByGroup.containsKey(svcGroupId) ? svcGroupId : processGroup.getIdentifier();
+                            svc.setParentGroupId(destinationGroupId);
+                            controllerServices.add(svc);
+                        });
                 }
             }
         }
@@ -203,7 +247,7 @@ public final class SnippetUtils {
                 final ProcessGroupDTO childGroupDto = dtoFactory.createProcessGroupDto(childGroup, recurse);
                 processGroups.add(childGroupDto);
 
-                addControllerServices(childGroup, childGroupDto, allServicesReferenced);
+                addControllerServices(childGroup, childGroupDto, allServicesReferenced, contentsByGroup, processGroup.getIdentifier());
             }
         }
 
@@ -219,6 +263,7 @@ public final class SnippetUtils {
             }
         }
 
+
         // Normalize the coordinates based on the locations of the other components
         final List<? extends ComponentDTO> components = new ArrayList<>();
         components.addAll((Set) processors);
@@ -231,6 +276,13 @@ public final class SnippetUtils {
         components.addAll((Set) remoteProcessGroups);
         normalizeCoordinates(components);
 
+        Set<ControllerServiceDTO> updatedControllerServices = snippetDto.getControllerServices();
+        if (updatedControllerServices == null) {
+            updatedControllerServices = new HashSet<>();
+        }
+        updatedControllerServices.addAll(controllerServices);
+        snippetDto.setControllerServices(updatedControllerServices);
+
         snippetDto.setProcessors(processors);
         snippetDto.setConnections(connections);
         snippetDto.setFunnels(funnels);
@@ -240,28 +292,50 @@ public final class SnippetUtils {
         snippetDto.setProcessGroups(processGroups);
         snippetDto.setRemoteProcessGroups(remoteProcessGroups);
 
-        snippetDto.setControllerServices(controllerServices);
-
         return snippetDto;
     }
 
-    private void addControllerServices(final ProcessGroup group, final ProcessGroupDTO dto, final Set<ControllerServiceDTO> allServicesReferenced) {
+    /**
+     * Finds all Controller Services that are referenced in the given Process Group (and child Process Groups, recursively), and
+     * adds them to the given servicesByGroup map
+     *
+     * @param group the Process Group to start from
+     * @param dto the DTO representation of the Process Group
+     * @param allServicesReferenced a Set of all Controller Service DTO's that have already been referenced; used to dedupe services
+     * @param contentsByGroup a Map of Process Group ID to the Process Group's contents
+     * @param highestGroupId the UUID of the 'highest' process group in the snippet
+     */
+    private void addControllerServices(final ProcessGroup group, final ProcessGroupDTO dto, final Set<ControllerServiceDTO> allServicesReferenced,
+        final Map<String, FlowSnippetDTO> contentsByGroup, final String highestGroupId) {
+
         final FlowSnippetDTO contents = dto.getContents();
+        contentsByGroup.put(dto.getId(), contents);
         if (contents == null) {
             return;
         }
 
-        final Set<ControllerServiceDTO> controllerServices = new HashSet<>();
 
         for (final ProcessorNode procNode : group.getProcessors()) {
             // Include all referenced services that are not already included in this snippet.
             getControllerServices(procNode.getProperties()).stream()
                 .filter(svc -> allServicesReferenced.add(svc))
-                .forEach(svc -> controllerServices.add(svc));
+                .forEach(svc -> {
+                    final String svcGroupId = svc.getParentGroupId();
+                    final String destinationGroupId = contentsByGroup.containsKey(svcGroupId) ? svcGroupId : highestGroupId;
+                    svc.setParentGroupId(destinationGroupId);
+                    final FlowSnippetDTO snippetDto = contentsByGroup.get(destinationGroupId);
+                    if (snippetDto != null) {
+                        Set<ControllerServiceDTO> services = snippetDto.getControllerServices();
+                        if (services == null) {
+                            snippetDto.setControllerServices(Collections.singleton(svc));
+                        } else {
+                            services.add(svc);
+                            snippetDto.setControllerServices(services);
+                        }
+                    }
+                });
         }
 
-        contents.setControllerServices(controllerServices);
-
         // Map child process group ID to the child process group for easy lookup
         final Map<String, ProcessGroupDTO> childGroupMap = contents.getProcessGroups().stream()
             .collect(Collectors.toMap(childGroupDto -> childGroupDto.getId(), childGroupDto -> childGroupDto));
@@ -272,7 +346,7 @@ public final class SnippetUtils {
                 continue;
             }
 
-            addControllerServices(childGroup, childDto, allServicesReferenced);
+            addControllerServices(childGroup, childDto, allServicesReferenced, contentsByGroup, highestGroupId);
         }
     }
 


[08/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
new file mode 100644
index 0000000..e7d5cb7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.HashSet;
+import java.util.Set;
+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;
+
+public class InFlightMessageTracker {
+    private final ConcurrentMap<FlowFile, Counts> messageCountsByFlowFile = new ConcurrentHashMap<>();
+    private final ConcurrentMap<FlowFile, Exception> failures = new ConcurrentHashMap<>();
+    private final Object progressMutex = new Object();
+
+    public void incrementAcknowledgedCount(final FlowFile flowFile) {
+        final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
+        counter.incrementAcknowledgedCount();
+
+        synchronized (progressMutex) {
+            progressMutex.notify();
+        }
+    }
+
+    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);
+
+        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 Collection<FlowFile> getSuccessfulFlowFiles() {
+                if (failures.isEmpty()) {
+                    return messageCountsByFlowFile.keySet();
+                }
+
+                final Set<FlowFile> flowFiles = new HashSet<>(messageCountsByFlowFile.keySet());
+                flowFiles.removeAll(failures.keySet());
+                return flowFiles;
+            }
+
+            @Override
+            public Collection<FlowFile> getFailedFlowFiles() {
+                return failures.keySet();
+            }
+
+            @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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index c74ad18..3d09f2d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -27,8 +27,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
-import org.apache.kafka.clients.CommonClientConfigs;
 
+import org.apache.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;
@@ -55,6 +56,10 @@ final class KafkaProcessorUtils {
 
     private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*";
 
+    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";
@@ -96,7 +101,6 @@ final class KafkaProcessorUtils {
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .expressionLanguageSupported(false)
             .build();
-
     static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
             .name("ssl.context.service")
             .displayName("SSL Context Service")
@@ -183,7 +187,7 @@ final class KafkaProcessorUtils {
 
         final Class<?> classType;
 
-        public KafkaConfigValidator(final Class classType) {
+        public KafkaConfigValidator(final Class<?> classType) {
             this.classType = classType;
         }
 
@@ -208,7 +212,8 @@ final class KafkaProcessorUtils {
         return builder.toString();
     }
 
-    static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map<String, String> mapToPopulate) {
+
+    static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
         for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
             if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
                 // Translate SSLContext Service configuration into Kafka properties
@@ -228,28 +233,32 @@ final class KafkaProcessorUtils {
                 }
             }
 
-            String pName = propertyDescriptor.getName();
-            String pValue = propertyDescriptor.isExpressionLanguageSupported()
+            String propertyName = propertyDescriptor.getName();
+            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
                     ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
                     : context.getProperty(propertyDescriptor).getValue();
-            if (pValue != null) {
-                if (pName.endsWith(".ms")) { // kafka standard time notation
-                    pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS));
+
+            if (propertyValue != null) {
+                // 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(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(pName, pValue);
+
+                if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
+                    mapToPopulate.put(propertyName, propertyValue);
                 }
             }
         }
     }
 
-    private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) {
+    private static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
         return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
     }
 
-    private static Set<String> getPublicStaticStringFieldValues(final Class... classes) {
+    private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
         final Set<String> strings = new HashSet<>();
-        for (final Class classType : classes) {
+        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 {

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
deleted file mode 100644
index 31a084f..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.io.Closeable;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.stream.io.util.StreamDemarcator;
-
-/**
- * Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
- * with sending contents of the {@link FlowFile}s to Kafka.
- */
-class KafkaPublisher implements Closeable {
-
-    private final Producer<byte[], byte[]> kafkaProducer;
-
-    private volatile long ackWaitTime = 30000;
-
-    private final ComponentLog componentLog;
-
-    private final int ackCheckSize;
-
-    KafkaPublisher(Properties kafkaProperties, ComponentLog componentLog) {
-        this(kafkaProperties, 100, componentLog);
-    }
-
-    /**
-     * Creates an instance of this class as well as the instance of the
-     * corresponding Kafka {@link KafkaProducer} using provided Kafka
-     * configuration properties.
-     *
-     * @param kafkaProperties instance of {@link Properties} used to bootstrap
-     * {@link KafkaProducer}
-     */
-    KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) {
-        this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
-        this.ackCheckSize = ackCheckSize;
-        this.componentLog = componentLog;
-    }
-
-    /**
-     * Publishes messages to Kafka topic. It uses {@link StreamDemarcator} to
-     * determine how many messages to Kafka will be sent from a provided
-     * {@link InputStream} (see {@link PublishingContext#getContentStream()}).
-     * It supports two publishing modes:
-     * <ul>
-     * <li>Sending all messages constructed from
-     * {@link StreamDemarcator#nextToken()} operation.</li>
-     * <li>Sending only unacknowledged messages constructed from
-     * {@link StreamDemarcator#nextToken()} operation.</li>
-     * </ul>
-     * The unacknowledged messages are determined from the value of
-     * {@link PublishingContext#getLastAckedMessageIndex()}.
-     * <br>
-     * This method assumes content stream affinity where it is expected that the
-     * content stream that represents the same Kafka message(s) will remain the
-     * same across possible retries. This is required specifically for cases
-     * where delimiter is used and a single content stream may represent
-     * multiple Kafka messages. The
-     * {@link PublishingContext#getLastAckedMessageIndex()} will provide the
-     * index of the last ACKed message, so upon retry only messages with the
-     * higher index are sent.
-     *
-     * @param publishingContext instance of {@link PublishingContext} which hold
-     * context information about the message(s) to be sent.
-     * @return The index of the last successful offset.
-     */
-    KafkaPublisherResult publish(PublishingContext publishingContext) {
-        StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
-                publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());
-
-        int prevLastAckedMessageIndex = publishingContext.getLastAckedMessageIndex();
-        List<Future<RecordMetadata>> resultFutures = new ArrayList<>();
-
-        byte[] messageBytes;
-        int tokenCounter = 0;
-        boolean continueSending = true;
-        KafkaPublisherResult result = null;
-        for (; continueSending && (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) {
-            if (prevLastAckedMessageIndex < tokenCounter) {
-                ProducerRecord<byte[], byte[]> message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes);
-                resultFutures.add(this.kafkaProducer.send(message));
-
-                if (tokenCounter % this.ackCheckSize == 0) {
-                    int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
-                    resultFutures.clear();
-                    if (lastAckedMessageIndex % this.ackCheckSize != 0) {
-                        continueSending = false;
-                        result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
-                    }
-                    prevLastAckedMessageIndex = lastAckedMessageIndex;
-                }
-            }
-        }
-
-        if (result == null) {
-            int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
-            resultFutures.clear();
-            result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
-        }
-        return result;
-    }
-
-    /**
-     * Sets the time this publisher will wait for the {@link Future#get()}
-     * operation (the Future returned by
-     * {@link KafkaProducer#send(ProducerRecord)}) to complete before timing
-     * out.
-     *
-     * This value will also be used as a timeout when closing the underlying
-     * {@link KafkaProducer}. See {@link #close()}.
-     */
-    void setAckWaitTime(long ackWaitTime) {
-        this.ackWaitTime = ackWaitTime;
-    }
-
-    /**
-     * This operation will process ACKs from Kafka in the order in which
-     * {@link KafkaProducer#send(ProducerRecord)} invocation were made returning
-     * the index of the last ACKed message. Within this operation processing ACK
-     * simply means successful invocation of 'get()' operation on the
-     * {@link Future} returned by {@link KafkaProducer#send(ProducerRecord)}
-     * operation. Upon encountering any type of error while interrogating such
-     * {@link Future} the ACK loop will end. Messages that were not ACKed would
-     * be considered non-delivered and therefore could be resent at the later
-     * time.
-     *
-     * @param sendFutures list of {@link Future}s representing results of
-     * publishing to Kafka
-     *
-     * @param lastAckMessageIndex the index of the last ACKed message. It is
-     * important to provide the last ACKed message especially while re-trying so
-     * the proper index is maintained.
-     */
-    private int processAcks(List<Future<RecordMetadata>> sendFutures, int lastAckMessageIndex) {
-        boolean exceptionThrown = false;
-        for (int segmentCounter = 0; segmentCounter < sendFutures.size() && !exceptionThrown; segmentCounter++) {
-            Future<RecordMetadata> future = sendFutures.get(segmentCounter);
-            try {
-                future.get(this.ackWaitTime, TimeUnit.MILLISECONDS);
-                lastAckMessageIndex++;
-            } catch (InterruptedException e) {
-                exceptionThrown = true;
-                Thread.currentThread().interrupt();
-                this.warnOrError("Interrupted while waiting for acks from Kafka", null);
-            } catch (ExecutionException e) {
-                exceptionThrown = true;
-                this.warnOrError("Failed while waiting for acks from Kafka", e);
-            } catch (TimeoutException e) {
-                exceptionThrown = true;
-                this.warnOrError("Timed out while waiting for acks from Kafka", null);
-            }
-        }
-
-        return lastAckMessageIndex;
-    }
-
-    /**
-     * Will close the underlying {@link KafkaProducer} waiting if necessary for
-     * the same duration as supplied {@link #setAckWaitTime(long)}
-     */
-    @Override
-    public void close() {
-        this.kafkaProducer.close(this.ackWaitTime, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     *
-     */
-    private void warnOrError(String message, Exception e) {
-        if (e == null) {
-            this.componentLog.warn(message);
-        } else {
-            this.componentLog.error(message, e);
-        }
-    }
-
-    /**
-     * Encapsulates the result received from publishing messages to Kafka
-     */
-    static class KafkaPublisherResult {
-
-        private final int messagesSent;
-        private final int lastMessageAcked;
-
-        KafkaPublisherResult(int messagesSent, int lastMessageAcked) {
-            this.messagesSent = messagesSent;
-            this.lastMessageAcked = lastMessageAcked;
-        }
-
-        public int getMessagesSent() {
-            return this.messagesSent;
-        }
-
-        public int getLastMessageAcked() {
-            return this.lastMessageAcked;
-        }
-
-        public boolean isAllAcked() {
-            return this.lastMessageAcked > -1 && this.messagesSent - 1 == this.lastMessageAcked;
-        }
-
-        @Override
-        public String toString() {
-            return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
index 4e1403d..af171bb 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.io.Closeable;
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
@@ -27,17 +28,16 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
+
 import javax.xml.bind.DatatypeConverter;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
+
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -46,201 +46,192 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.9.x"})
-@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.9 producer. "
-        + "The messages to send may be individual FlowFiles or may be delimited, using a "
-        + "user-specified delimiter, such as a new-line. "
-        + " Please note there are cases where the publisher can get into an indefinite stuck state.  We are closely monitoring"
-        + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can.  In the mean time"
-        + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.")
+@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.9.x Producer. "
+    + "The messages to send may be individual FlowFiles or may be delimited, using a "
+    + "user-specified delimiter, such as a new-line. "
+    + " Please note there are cases where the publisher can get into an indefinite stuck state.  We are closely monitoring"
+    + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can.  In the mean time"
+    + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on. The complementary NiFi processor for fetching messages is ConsumeKafka.")
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
-        description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+    description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
         + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
-public class PublishKafka extends AbstractSessionFactoryProcessor {
-
-    private final Logger logger = LoggerFactory.getLogger(this.getClass());
-
-    protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id";
-
-    protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx";
-
-    protected static final String FAILED_TOPIC_ATTR = "failed.topic";
-
-    protected static final String FAILED_KEY_ATTR = "failed.key";
-
-    protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter";
-
+@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 <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+    + "be greater than 1.")
+public class PublishKafka 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 "
+        "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, "
+        "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 <Guarantee Replicated Delivery> "
             + "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, "
+        "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, "
+        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.");
+        "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();
+        .name("topic")
+        .displayName("Topic Name")
+        .description("The name of the Kafka Topic to publish to.")
+        .required(true)
+        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.ACKS_CONFIG)
-            .displayName("Delivery Guarantee")
-            .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
-            .required(true)
-            .expressionLanguageSupported(false)
-            .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
-            .defaultValue(DELIVERY_BEST_EFFORT.getValue())
-            .build();
-
-    static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.MAX_BLOCK_MS_CONFIG)
-            .displayName("Meta Data Wait Time")
-            .description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the "
-                    + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
-            .required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .defaultValue("30 sec")
-            .build();
+        .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();
+        .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();
+        .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();
+        .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();
+        .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();
+        .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();
+        .name(ProducerConfig.COMPRESSION_TYPE_CONFIG)
+        .displayName("Compression Type")
+        .description("This parameter allows you to specify the compression codec for all data generated by this producer.")
+        .required(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .allowableValues("none", "gzip", "snappy", "lz4")
+        .defaultValue("none")
+        .build();
 
     static final Relationship REL_SUCCESS = new Relationship.Builder()
-            .name("success")
-            .description("FlowFiles for which all content was sent to Kafka.")
-            .build();
+        .name("success")
+        .description("FlowFiles for which all content was sent to Kafka.")
+        .build();
 
     static final Relationship REL_FAILURE = new Relationship.Builder()
-            .name("failure")
-            .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
-            .build();
-
-    static final List<PropertyDescriptor> DESCRIPTORS;
-
-    static final Set<Relationship> RELATIONSHIPS;
-
-    private volatile String brokers;
+        .name("failure")
+        .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
+        .build();
 
-    private final AtomicInteger taskCounter = new AtomicInteger();
+    private static final List<PropertyDescriptor> PROPERTIES;
+    private static final Set<Relationship> RELATIONSHIPS;
 
-    private volatile boolean acceptTask = true;
+    private volatile PublisherPool publisherPool = null;
 
-    /*
-     * Will ensure that list of PropertyDescriptors is build only once, since
-     * all other lifecycle methods are invoked multiple times.
-     */
     static {
-        final List<PropertyDescriptor> _descriptors = new ArrayList<>();
-        _descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
-        _descriptors.add(TOPIC);
-        _descriptors.add(DELIVERY_GUARANTEE);
-        _descriptors.add(KEY);
-        _descriptors.add(KEY_ATTRIBUTE_ENCODING);
-        _descriptors.add(MESSAGE_DEMARCATOR);
-        _descriptors.add(MAX_REQUEST_SIZE);
-        _descriptors.add(META_WAIT_TIME);
-        _descriptors.add(PARTITION_CLASS);
-        _descriptors.add(COMPRESSION_CODEC);
-
-        DESCRIPTORS = Collections.unmodifiableList(_descriptors);
-
-        final Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        RELATIONSHIPS = Collections.unmodifiableSet(_relationships);
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        properties.add(TOPIC);
+        properties.add(DELIVERY_GUARANTEE);
+        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<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
     }
 
     @Override
@@ -250,15 +241,17 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return DESCRIPTORS;
+        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();
+            .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
+            .name(propertyDescriptorName)
+            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .dynamic(true)
+            .build();
     }
 
     @Override
@@ -266,226 +259,123 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
         return KafkaProcessorUtils.validateCommonProperties(validationContext);
     }
 
-    volatile KafkaPublisher kafkaPublisher;
-
-    /**
-     * This thread-safe operation will delegate to
-     * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first
-     * checking and creating (if necessary) Kafka resource which could be either
-     * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and
-     * destroy the underlying Kafka resource upon catching an {@link Exception}
-     * raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}.
-     * After Kafka resource is destroyed it will be re-created upon the next
-     * invocation of this operation essentially providing a self healing
-     * mechanism to deal with potentially corrupted resource.
-     * <p>
-     * Keep in mind that upon catching an exception the state of this processor
-     * will be set to no longer accept any more tasks, until Kafka resource is
-     * reset. This means that in a multi-threaded situation currently executing
-     * tasks will be given a chance to complete while no new tasks will be
-     * accepted.
-     *
-     * @param context context
-     * @param sessionFactory factory
-     */
-    @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted.
-            this.taskCounter.incrementAndGet();
-            final ProcessSession session = sessionFactory.createSession();
-            try {
-                /*
-                 * We can't be doing double null check here since as a pattern
-                 * it only works for lazy init but not reset, which is what we
-                 * are doing here. In fact the first null check is dangerous
-                 * since 'kafkaPublisher' can become null right after its null
-                 * check passed causing subsequent NPE.
-                 */
-                synchronized (this) {
-                    if (this.kafkaPublisher == null) {
-                        this.kafkaPublisher = this.buildKafkaResource(context, session);
-                    }
-                }
-
-                /*
-                 * The 'processed' boolean flag does not imply any failure or success. It simply states that:
-                 * - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated
-                 * - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile
-                 */
-                boolean processed = this.rendezvousWithKafka(context, session);
-                session.commit();
-                if (!processed) {
-                    context.yield();
-                }
-            } catch (Throwable e) {
-                this.acceptTask = false;
-                session.rollback(true);
-                this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e});
-            } finally {
-                synchronized (this) {
-                    if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) {
-                        this.close();
-                        this.acceptTask = true;
-                    }
-                }
-            }
-        } else {
-            this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
-            this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
-            context.yield();
+    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 Map<String, Object> 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);
     }
 
-    /**
-     * Will call {@link Closeable#close()} on the target resource after which
-     * the target resource will be set to null. Should only be called when there
-     * are no more threads being executed on this processor or when it has been
-     * verified that only a single thread remains.
-     *
-     * @see KafkaPublisher
-     * @see KafkaConsumer
-     */
     @OnStopped
-    public void close() {
-        try {
-            if (this.kafkaPublisher != null) {
-                try {
-                    this.kafkaPublisher.close();
-                } catch (Exception e) {
-                    this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e);
-                }
-            }
-        } finally {
-            this.kafkaPublisher = null;
+    public void closePool() {
+        if (publisherPool != null) {
+            publisherPool.close();
         }
+
+        publisherPool = null;
     }
 
-    /**
-     * Will rendezvous with Kafka if {@link ProcessSession} contains
-     * {@link FlowFile} producing a result {@link FlowFile}.
-     * <br>
-     * The result {@link FlowFile} that is successful is then transfered to
-     * {@link #REL_SUCCESS}
-     * <br>
-     * The result {@link FlowFile} that is failed is then transfered to
-     * {@link #REL_FAILURE}
-     *
-     */
-    protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) {
-        FlowFile flowFile = session.get();
-        if (flowFile != null) {
-            long start = System.nanoTime();
-            flowFile = this.doRendezvousWithKafka(flowFile, context, session);
-            Relationship relationship = REL_SUCCESS;
-            if (!this.isFailedFlowFile(flowFile)) {
-                String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
-                long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-                String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic);
-                session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration);
-                this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis",
-                        new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration});
-            } else {
-                relationship = REL_FAILURE;
-                flowFile = session.penalize(flowFile);
-            }
-            session.transfer(flowFile, relationship);
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet();
+
+        final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500));
+        if (flowFiles.isEmpty()) {
+            return;
         }
-        return flowFile != null;
-    }
 
-    /**
-     * Builds and instance of {@link KafkaPublisher}.
-     */
-    protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) {
-        final Map<String, String> kafkaProps = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps);
-        kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProps.put("max.request.size", String.valueOf(context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue()));
-        this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
-        final Properties props = new Properties();
-        props.putAll(kafkaProps);
-        KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger());
-        return publisher;
-    }
+        final PublisherPool pool = getPublisherPool(context);
+        if (pool == null) {
+            context.yield();
+            return;
+        }
 
-    /**
-     * Will rendezvous with {@link KafkaPublisher} after building
-     * {@link PublishingContext} and will produce the resulting
-     * {@link FlowFile}. The resulting FlowFile contains all required
-     * information to determine if message publishing originated from the
-     * provided FlowFile has actually succeeded fully, partially or failed
-     * completely (see {@link #isFailedFlowFile(FlowFile)}.
-     */
-    private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
-        final AtomicReference<KafkaPublisher.KafkaPublisherResult> publishResultRef = new AtomicReference<>();
-        session.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(InputStream contentStream) throws IOException {
-                PublishingContext publishingContext = PublishKafka.this.buildPublishingContext(flowFile, context, contentStream);
-                KafkaPublisher.KafkaPublisherResult result = PublishKafka.this.kafkaPublisher.publish(publishingContext);
-                publishResultRef.set(result);
+        final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
+        final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
+
+        final long startTime = System.nanoTime();
+        try (final PublisherLease lease = pool.obtainPublisher()) {
+            // Send each FlowFile to Kafka asynchronously.
+            for (final FlowFile flowFile : flowFiles) {
+                if (!isScheduled()) {
+                    // If stopped, re-queue FlowFile instead of sending it
+                    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);
+                        }
+                    }
+                });
             }
-        });
 
-        FlowFile resultFile = publishResultRef.get().isAllAcked()
-                ? this.cleanUpFlowFileIfNecessary(flowFile, session)
-                : session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context));
+            // Complete the send
+            final PublishResult publishResult = lease.complete();
 
-        if (!this.isFailedFlowFile(resultFile)) {
-            resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent()));
-        }
-        return resultFile;
-    }
+            // Transfer any successful FlowFiles.
+            final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
+            for (FlowFile success : publishResult.getSuccessfulFlowFiles()) {
+                final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue();
 
-    /**
-     * Builds {@link PublishingContext} for message(s) to be sent to Kafka.
-     * {@link PublishingContext} contains all contextual information required by
-     * {@link KafkaPublisher} to publish to Kafka. Such information contains
-     * things like topic name, content stream, delimiter, key and last ACKed
-     * message for cases where provided FlowFile is being retried (failed in the
-     * past).
-     * <br>
-     * For the clean FlowFile (file that has been sent for the first time),
-     * PublishingContext will be built form {@link ProcessContext} associated
-     * with this invocation.
-     * <br>
-     * For the failed FlowFile, {@link PublishingContext} will be built from
-     * attributes of that FlowFile which by then will already contain required
-     * information (e.g., topic, key, delimiter etc.). This is required to
-     * ensure the affinity of the retry in the even where processor
-     * configuration has changed. However keep in mind that failed FlowFile is
-     * only considered a failed FlowFile if it is being re-processed by the same
-     * processor (determined via {@link #FAILED_PROC_ID_ATTR}, see
-     * {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to
-     * another PublishKafka0_10 processor it is treated as a fresh FlowFile
-     * regardless if it has #FAILED* attributes set.
-     */
-    private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) {
-        final byte[] keyBytes = getMessageKey(flowFile, context);
-
-        final String topicName;
-        final byte[] delimiterBytes;
-        int lastAckedMessageIndex = -1;
-        if (this.isFailedFlowFile(flowFile)) {
-            lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
-            topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
-            delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
-                    ? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
-        } else {
-            topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
-            delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR)
-                    .evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
-        }
+                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);
+            }
+
+            // Transfer any failures.
+            for (final FlowFile failure : publishResult.getFailedFlowFiles()) {
+                final int successCount = publishResult.getSuccessfulMessageCount(failure);
+                if (successCount > 0) {
+                    getLogger().error("Failed to send some messages for {} to Kafka, but {} messages were acknowledged by Kafka. Routing to failure due to {}",
+                        new Object[] {failure, successCount, publishResult.getReasonForFailure(failure)});
+                } else {
+                    getLogger().error("Failed to send all message for {} to Kafka; routing to failure due to {}",
+                        new Object[] {failure, publishResult.getReasonForFailure(failure)});
+                }
 
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex,
-                context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue());
-        publishingContext.setKeyBytes(keyBytes);
-        publishingContext.setDelimiterBytes(delimiterBytes);
-        return publishingContext;
+                session.transfer(failure, REL_FAILURE);
+            }
+        }
     }
 
+
     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();
@@ -504,51 +394,4 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
 
         return DatatypeConverter.parseHexBinary(uninterpretedKey);
     }
-
-    /**
-     * Will remove FAILED_* attributes if FlowFile is no longer considered a
-     * failed FlowFile
-     *
-     * @see #isFailedFlowFile(FlowFile)
-     */
-    private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) {
-        if (this.isFailedFlowFile(flowFile)) {
-            Set<String> keysToRemove = new HashSet<>();
-            keysToRemove.add(FAILED_DELIMITER_ATTR);
-            keysToRemove.add(FAILED_KEY_ATTR);
-            keysToRemove.add(FAILED_TOPIC_ATTR);
-            keysToRemove.add(FAILED_PROC_ID_ATTR);
-            keysToRemove.add(FAILED_LAST_ACK_IDX);
-            flowFile = session.removeAllAttributes(flowFile, keysToRemove);
-        }
-        return flowFile;
-    }
-
-    /**
-     * Builds a {@link Map} of FAILED_* attributes
-     *
-     * @see #FAILED_PROC_ID_ATTR
-     * @see #FAILED_LAST_ACK_IDX
-     * @see #FAILED_TOPIC_ATTR
-     * @see #FAILED_KEY_ATTR
-     * @see #FAILED_DELIMITER_ATTR
-     */
-    private Map<String, String> buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile, ProcessContext context) {
-        Map<String, String> attributes = new HashMap<>();
-        attributes.put(FAILED_PROC_ID_ATTR, this.getIdentifier());
-        attributes.put(FAILED_LAST_ACK_IDX, String.valueOf(lastAckedMessageIndex));
-        attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue());
-        attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue());
-        attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet()
-                ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null);
-        return attributes;
-    }
-
-    /**
-     * Returns 'true' if provided FlowFile is a failed FlowFile. A failed
-     * FlowFile contains {@link #FAILED_PROC_ID_ATTR}.
-     */
-    private boolean isFailedFlowFile(FlowFile flowFile) {
-        return this.getIdentifier().equals(flowFile.getAttribute(FAILED_PROC_ID_ATTR));
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
new file mode 100644
index 0000000..b685265
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
@@ -0,0 +1,56 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+
+import org.apache.nifi.flowfile.FlowFile;
+
+public interface PublishResult {
+    Collection<FlowFile> getSuccessfulFlowFiles();
+
+    Collection<FlowFile> getFailedFlowFiles();
+
+    int getSuccessfulMessageCount(FlowFile flowFile);
+
+    Exception getReasonForFailure(FlowFile flowFile);
+
+
+    public static final PublishResult EMPTY = new PublishResult() {
+        @Override
+        public Collection<FlowFile> getSuccessfulFlowFiles() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public Collection<FlowFile> getFailedFlowFiles() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public int getSuccessfulMessageCount(FlowFile flowFile) {
+            return 0;
+        }
+
+        @Override
+        public Exception getReasonForFailure(FlowFile flowFile) {
+            return null;
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
new file mode 100644
index 0000000..b67e8a8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+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.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+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<byte[], byte[]> producer;
+    private final int maxMessageSize;
+    private final long maxAckWaitMillis;
+    private volatile boolean poisoned = false;
+
+    private InFlightMessageTracker tracker;
+
+    public PublisherLease(final Producer<byte[], byte[]> producer, final int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
+        this.producer = producer;
+        this.maxMessageSize = maxMessageSize;
+        this.logger = logger;
+        this.maxAckWaitMillis = maxAckWaitMillis;
+    }
+
+    protected void poison() {
+        this.poisoned = true;
+    }
+
+    public boolean isPoisoned() {
+        return poisoned;
+    }
+
+    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();
+        }
+
+        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;
+        }
+    }
+
+    private void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker) {
+        final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, null, messageKey, messageContent);
+        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();
+                }
+            }
+        });
+
+        tracker.incrementSentCount(flowFile);
+    }
+
+    public PublishResult complete() {
+        if (tracker == null) {
+            throw new IllegalStateException("Cannot complete publishing to Kafka because Publisher Lease was already closed");
+        }
+
+        producer.flush();
+
+        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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
new file mode 100644
index 0000000..5902b03
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+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<PublisherLease> publisherQueue;
+    private final Map<String, Object> kafkaProperties;
+    private final int maxMessageSize;
+    private final long maxAckWaitMillis;
+
+    private volatile boolean closed = false;
+
+    PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis) {
+        this.logger = logger;
+        this.publisherQueue = new LinkedBlockingQueue<>();
+        this.kafkaProperties = kafkaProperties;
+        this.maxMessageSize = maxMessageSize;
+        this.maxAckWaitMillis = maxAckWaitMillis;
+    }
+
+    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 Producer<byte[], byte[]> producer = new KafkaProducer<>(kafkaProperties);
+        final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger) {
+            @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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
deleted file mode 100644
index 1513481..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-
-/**
- * Holder of context information used by {@link KafkaPublisher} required to
- * publish messages to Kafka.
- */
-class PublishingContext {
-
-    private final InputStream contentStream;
-
-    private final String topic;
-
-    private final int lastAckedMessageIndex;
-
-    private final int maxRequestSize;
-
-    private byte[] keyBytes;
-
-    private byte[] delimiterBytes;
-
-    PublishingContext(InputStream contentStream, String topic) {
-        this(contentStream, topic, -1);
-    }
-
-    PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
-        this(contentStream, topic, lastAckedMessageIndex, 1048576);
-    }
-
-    PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex, int maxRequestSize) {
-        this.validateInput(contentStream, topic, lastAckedMessageIndex);
-        this.contentStream = contentStream;
-        this.topic = topic;
-        this.lastAckedMessageIndex = lastAckedMessageIndex;
-        this.maxRequestSize = maxRequestSize;
-    }
-
-    @Override
-    public String toString() {
-        return "topic: '" + this.topic + "'; delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'";
-    }
-
-    int getLastAckedMessageIndex() {
-        return this.lastAckedMessageIndex;
-    }
-
-    int getMaxRequestSize() {
-        return this.maxRequestSize;
-    }
-
-    byte[] getKeyBytes() {
-        return this.keyBytes;
-    }
-
-    byte[] getDelimiterBytes() {
-        return this.delimiterBytes;
-    }
-
-    InputStream getContentStream() {
-        return this.contentStream;
-    }
-
-    String getTopic() {
-        return this.topic;
-    }
-
-    void setKeyBytes(byte[] keyBytes) {
-        if (this.keyBytes == null) {
-            if (keyBytes != null) {
-                this.assertBytesValid(keyBytes);
-                this.keyBytes = keyBytes;
-            }
-        } else {
-            throw new IllegalArgumentException("'keyBytes' can only be set once per instance");
-        }
-    }
-
-    void setDelimiterBytes(byte[] delimiterBytes) {
-        if (this.delimiterBytes == null) {
-            if (delimiterBytes != null) {
-                this.assertBytesValid(delimiterBytes);
-                this.delimiterBytes = delimiterBytes;
-            }
-        } else {
-            throw new IllegalArgumentException("'delimiterBytes' can only be set once per instance");
-        }
-    }
-
-    private void assertBytesValid(byte[] bytes) {
-        if (bytes != null) {
-            if (bytes.length == 0) {
-                throw new IllegalArgumentException("'bytes' must not be empty");
-            }
-        }
-    }
-
-    private void validateInput(InputStream contentStream, String topic, int lastAckedMessageIndex) {
-        if (contentStream == null) {
-            throw new IllegalArgumentException("'contentStream' must not be null");
-        } else if (topic == null || topic.trim().length() == 0) {
-            throw new IllegalArgumentException("'topic' must not be null or empty");
-        } else if (lastAckedMessageIndex < -1) {
-            throw new IllegalArgumentException("'lastAckedMessageIndex' must be >= -1");
-        }
-    }
-}


[11/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
new file mode 100644
index 0000000..e7d5cb7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/InFlightMessageTracker.java
@@ -0,0 +1,178 @@
+/*
+ * 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.Collection;
+import java.util.HashSet;
+import java.util.Set;
+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;
+
+public class InFlightMessageTracker {
+    private final ConcurrentMap<FlowFile, Counts> messageCountsByFlowFile = new ConcurrentHashMap<>();
+    private final ConcurrentMap<FlowFile, Exception> failures = new ConcurrentHashMap<>();
+    private final Object progressMutex = new Object();
+
+    public void incrementAcknowledgedCount(final FlowFile flowFile) {
+        final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
+        counter.incrementAcknowledgedCount();
+
+        synchronized (progressMutex) {
+            progressMutex.notify();
+        }
+    }
+
+    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);
+
+        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 Collection<FlowFile> getSuccessfulFlowFiles() {
+                if (failures.isEmpty()) {
+                    return messageCountsByFlowFile.keySet();
+                }
+
+                final Set<FlowFile> flowFiles = new HashSet<>(messageCountsByFlowFile.keySet());
+                flowFiles.removeAll(failures.keySet());
+                return flowFiles;
+            }
+
+            @Override
+            public Collection<FlowFile> getFailedFlowFiles() {
+                return failures.keySet();
+            }
+
+            @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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index 3ae7495..3d09f2d 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -27,8 +27,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
-import org.apache.kafka.clients.CommonClientConfigs;
 
+import org.apache.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;
@@ -55,6 +56,10 @@ final class KafkaProcessorUtils {
 
     private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*";
 
+    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";
@@ -182,7 +187,7 @@ final class KafkaProcessorUtils {
 
         final Class<?> classType;
 
-        public KafkaConfigValidator(final Class classType) {
+        public KafkaConfigValidator(final Class<?> classType) {
             this.classType = classType;
         }
 
@@ -207,7 +212,8 @@ final class KafkaProcessorUtils {
         return builder.toString();
     }
 
-    static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map<String, String> mapToPopulate) {
+
+    static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
         for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
             if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
                 // Translate SSLContext Service configuration into Kafka properties
@@ -226,28 +232,33 @@ final class KafkaProcessorUtils {
                     mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
                 }
             }
-            String pName = propertyDescriptor.getName();
-            String pValue = propertyDescriptor.isExpressionLanguageSupported()
+
+            String propertyName = propertyDescriptor.getName();
+            String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
                     ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
                     : context.getProperty(propertyDescriptor).getValue();
-            if (pValue != null) {
-                if (pName.endsWith(".ms")) { // kafka standard time notation
-                    pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS));
+
+            if (propertyValue != null) {
+                // 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(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
-                    mapToPopulate.put(pName, pValue);
+
+                if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
+                    mapToPopulate.put(propertyName, propertyValue);
                 }
             }
         }
     }
 
-    private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) {
+    private static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
         return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
     }
 
-    private static Set<String> getPublicStaticStringFieldValues(final Class... classes) {
+    private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
         final Set<String> strings = new HashSet<>();
-        for (final Class classType : classes) {
+        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 {

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
deleted file mode 100644
index 31a084f..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaPublisher.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.io.Closeable;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.stream.io.util.StreamDemarcator;
-
-/**
- * Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
- * with sending contents of the {@link FlowFile}s to Kafka.
- */
-class KafkaPublisher implements Closeable {
-
-    private final Producer<byte[], byte[]> kafkaProducer;
-
-    private volatile long ackWaitTime = 30000;
-
-    private final ComponentLog componentLog;
-
-    private final int ackCheckSize;
-
-    KafkaPublisher(Properties kafkaProperties, ComponentLog componentLog) {
-        this(kafkaProperties, 100, componentLog);
-    }
-
-    /**
-     * Creates an instance of this class as well as the instance of the
-     * corresponding Kafka {@link KafkaProducer} using provided Kafka
-     * configuration properties.
-     *
-     * @param kafkaProperties instance of {@link Properties} used to bootstrap
-     * {@link KafkaProducer}
-     */
-    KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) {
-        this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
-        this.ackCheckSize = ackCheckSize;
-        this.componentLog = componentLog;
-    }
-
-    /**
-     * Publishes messages to Kafka topic. It uses {@link StreamDemarcator} to
-     * determine how many messages to Kafka will be sent from a provided
-     * {@link InputStream} (see {@link PublishingContext#getContentStream()}).
-     * It supports two publishing modes:
-     * <ul>
-     * <li>Sending all messages constructed from
-     * {@link StreamDemarcator#nextToken()} operation.</li>
-     * <li>Sending only unacknowledged messages constructed from
-     * {@link StreamDemarcator#nextToken()} operation.</li>
-     * </ul>
-     * The unacknowledged messages are determined from the value of
-     * {@link PublishingContext#getLastAckedMessageIndex()}.
-     * <br>
-     * This method assumes content stream affinity where it is expected that the
-     * content stream that represents the same Kafka message(s) will remain the
-     * same across possible retries. This is required specifically for cases
-     * where delimiter is used and a single content stream may represent
-     * multiple Kafka messages. The
-     * {@link PublishingContext#getLastAckedMessageIndex()} will provide the
-     * index of the last ACKed message, so upon retry only messages with the
-     * higher index are sent.
-     *
-     * @param publishingContext instance of {@link PublishingContext} which hold
-     * context information about the message(s) to be sent.
-     * @return The index of the last successful offset.
-     */
-    KafkaPublisherResult publish(PublishingContext publishingContext) {
-        StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
-                publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());
-
-        int prevLastAckedMessageIndex = publishingContext.getLastAckedMessageIndex();
-        List<Future<RecordMetadata>> resultFutures = new ArrayList<>();
-
-        byte[] messageBytes;
-        int tokenCounter = 0;
-        boolean continueSending = true;
-        KafkaPublisherResult result = null;
-        for (; continueSending && (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) {
-            if (prevLastAckedMessageIndex < tokenCounter) {
-                ProducerRecord<byte[], byte[]> message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes);
-                resultFutures.add(this.kafkaProducer.send(message));
-
-                if (tokenCounter % this.ackCheckSize == 0) {
-                    int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
-                    resultFutures.clear();
-                    if (lastAckedMessageIndex % this.ackCheckSize != 0) {
-                        continueSending = false;
-                        result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
-                    }
-                    prevLastAckedMessageIndex = lastAckedMessageIndex;
-                }
-            }
-        }
-
-        if (result == null) {
-            int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
-            resultFutures.clear();
-            result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
-        }
-        return result;
-    }
-
-    /**
-     * Sets the time this publisher will wait for the {@link Future#get()}
-     * operation (the Future returned by
-     * {@link KafkaProducer#send(ProducerRecord)}) to complete before timing
-     * out.
-     *
-     * This value will also be used as a timeout when closing the underlying
-     * {@link KafkaProducer}. See {@link #close()}.
-     */
-    void setAckWaitTime(long ackWaitTime) {
-        this.ackWaitTime = ackWaitTime;
-    }
-
-    /**
-     * This operation will process ACKs from Kafka in the order in which
-     * {@link KafkaProducer#send(ProducerRecord)} invocation were made returning
-     * the index of the last ACKed message. Within this operation processing ACK
-     * simply means successful invocation of 'get()' operation on the
-     * {@link Future} returned by {@link KafkaProducer#send(ProducerRecord)}
-     * operation. Upon encountering any type of error while interrogating such
-     * {@link Future} the ACK loop will end. Messages that were not ACKed would
-     * be considered non-delivered and therefore could be resent at the later
-     * time.
-     *
-     * @param sendFutures list of {@link Future}s representing results of
-     * publishing to Kafka
-     *
-     * @param lastAckMessageIndex the index of the last ACKed message. It is
-     * important to provide the last ACKed message especially while re-trying so
-     * the proper index is maintained.
-     */
-    private int processAcks(List<Future<RecordMetadata>> sendFutures, int lastAckMessageIndex) {
-        boolean exceptionThrown = false;
-        for (int segmentCounter = 0; segmentCounter < sendFutures.size() && !exceptionThrown; segmentCounter++) {
-            Future<RecordMetadata> future = sendFutures.get(segmentCounter);
-            try {
-                future.get(this.ackWaitTime, TimeUnit.MILLISECONDS);
-                lastAckMessageIndex++;
-            } catch (InterruptedException e) {
-                exceptionThrown = true;
-                Thread.currentThread().interrupt();
-                this.warnOrError("Interrupted while waiting for acks from Kafka", null);
-            } catch (ExecutionException e) {
-                exceptionThrown = true;
-                this.warnOrError("Failed while waiting for acks from Kafka", e);
-            } catch (TimeoutException e) {
-                exceptionThrown = true;
-                this.warnOrError("Timed out while waiting for acks from Kafka", null);
-            }
-        }
-
-        return lastAckMessageIndex;
-    }
-
-    /**
-     * Will close the underlying {@link KafkaProducer} waiting if necessary for
-     * the same duration as supplied {@link #setAckWaitTime(long)}
-     */
-    @Override
-    public void close() {
-        this.kafkaProducer.close(this.ackWaitTime, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     *
-     */
-    private void warnOrError(String message, Exception e) {
-        if (e == null) {
-            this.componentLog.warn(message);
-        } else {
-            this.componentLog.error(message, e);
-        }
-    }
-
-    /**
-     * Encapsulates the result received from publishing messages to Kafka
-     */
-    static class KafkaPublisherResult {
-
-        private final int messagesSent;
-        private final int lastMessageAcked;
-
-        KafkaPublisherResult(int messagesSent, int lastMessageAcked) {
-            this.messagesSent = messagesSent;
-            this.lastMessageAcked = lastMessageAcked;
-        }
-
-        public int getMessagesSent() {
-            return this.messagesSent;
-        }
-
-        public int getLastMessageAcked() {
-            return this.lastMessageAcked;
-        }
-
-        public boolean isAllAcked() {
-            return this.lastMessageAcked > -1 && this.messagesSent - 1 == this.lastMessageAcked;
-        }
-
-        @Override
-        public String toString() {
-            return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
index 18f3018..bb0bed2 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.processors.kafka.pubsub;
 
-import java.io.Closeable;
+import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
@@ -27,17 +28,16 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
+
 import javax.xml.bind.DatatypeConverter;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
+
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -46,202 +46,192 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.10"})
-@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.10 producer. "
-        + "The messages to send may be individual FlowFiles or may be delimited, using a "
-        + "user-specified delimiter, such as a new-line. "
-        + " Please note there are cases where the publisher can get into an indefinite stuck state.  We are closely monitoring"
-        + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can.  In the mean time"
-        + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.")
+
+@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.10.x"})
+@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.10.x Producer API."
+    + "The messages to send may be individual FlowFiles or may be delimited, using a "
+    + "user-specified delimiter, such as a new-line. "
+    + " Please note there are cases where the publisher can get into an indefinite stuck state.  We are closely monitoring"
+    + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can.  In the 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 fetching messages is ConsumeKafka_0_10.")
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
-        description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+    description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
         + " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
-public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
-
-    private final Logger logger = LoggerFactory.getLogger(this.getClass());
-
-    protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id";
-
-    protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx";
-
-    protected static final String FAILED_TOPIC_ATTR = "failed.topic";
-
-    protected static final String FAILED_KEY_ATTR = "failed.key";
-
-    protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter";
-
+@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 <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+    + "be greater than 1.")
+public class PublishKafka_0_10 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 "
+        "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, "
+        "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 <Guarantee Replicated Delivery> "
             + "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, "
+        "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, "
+        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.");
+        "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();
+        .name("topic")
+        .displayName("Topic Name")
+        .description("The name of the Kafka Topic to publish to.")
+        .required(true)
+        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
 
     static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.ACKS_CONFIG)
-            .displayName("Delivery Guarantee")
-            .description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
-            .required(true)
-            .expressionLanguageSupported(false)
-            .allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
-            .defaultValue(DELIVERY_BEST_EFFORT.getValue())
-            .build();
-
-    static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder()
-            .name(ProducerConfig.MAX_BLOCK_MS_CONFIG)
-            .displayName("Meta Data Wait Time")
-            .description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the "
-                    + "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
-            .required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .defaultValue("30 sec")
-            .build();
+        .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();
+        .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.  It will be serialized as UTF-8 bytes. "
-                    + "If not specified then the flow file attribute kafka.key is used if present "
-                    + "and we're not demarcating. In that case the hex string is coverted to its byte"
-                    + "form and written as a byte[] key.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .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();
+        .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();
+        .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();
+        .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();
+        .name(ProducerConfig.COMPRESSION_TYPE_CONFIG)
+        .displayName("Compression Type")
+        .description("This parameter allows you to specify the compression codec for all data generated by this producer.")
+        .required(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .allowableValues("none", "gzip", "snappy", "lz4")
+        .defaultValue("none")
+        .build();
 
     static final Relationship REL_SUCCESS = new Relationship.Builder()
-            .name("success")
-            .description("FlowFiles for which all content was sent to Kafka.")
-            .build();
+        .name("success")
+        .description("FlowFiles for which all content was sent to Kafka.")
+        .build();
 
     static final Relationship REL_FAILURE = new Relationship.Builder()
-            .name("failure")
-            .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
-            .build();
-
-    static final List<PropertyDescriptor> DESCRIPTORS;
-
-    static final Set<Relationship> RELATIONSHIPS;
+        .name("failure")
+        .description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
+        .build();
 
-    private volatile String brokers;
+    private static final List<PropertyDescriptor> PROPERTIES;
+    private static final Set<Relationship> RELATIONSHIPS;
 
-    private final AtomicInteger taskCounter = new AtomicInteger();
+    private volatile PublisherPool publisherPool = null;
 
-    private volatile boolean acceptTask = true;
-
-    /*
-     * Will ensure that list of PropertyDescriptors is build only once, since
-     * all other lifecycle methods are invoked multiple times.
-     */
     static {
-        final List<PropertyDescriptor> _descriptors = new ArrayList<>();
-        _descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
-        _descriptors.add(TOPIC);
-        _descriptors.add(DELIVERY_GUARANTEE);
-        _descriptors.add(KEY);
-        _descriptors.add(KEY_ATTRIBUTE_ENCODING);
-        _descriptors.add(MESSAGE_DEMARCATOR);
-        _descriptors.add(MAX_REQUEST_SIZE);
-        _descriptors.add(META_WAIT_TIME);
-        _descriptors.add(PARTITION_CLASS);
-        _descriptors.add(COMPRESSION_CODEC);
-
-        DESCRIPTORS = Collections.unmodifiableList(_descriptors);
-
-        final Set<Relationship> _relationships = new HashSet<>();
-        _relationships.add(REL_SUCCESS);
-        _relationships.add(REL_FAILURE);
-        RELATIONSHIPS = Collections.unmodifiableSet(_relationships);
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        properties.add(TOPIC);
+        properties.add(DELIVERY_GUARANTEE);
+        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<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
     }
 
     @Override
@@ -251,15 +241,17 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return DESCRIPTORS;
+        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();
+            .description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
+            .name(propertyDescriptorName)
+            .addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+            .dynamic(true)
+            .build();
     }
 
     @Override
@@ -267,226 +259,123 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
         return KafkaProcessorUtils.validateCommonProperties(validationContext);
     }
 
-    volatile KafkaPublisher kafkaPublisher;
-
-    /**
-     * This thread-safe operation will delegate to
-     * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first
-     * checking and creating (if necessary) Kafka resource which could be either
-     * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and
-     * destroy the underlying Kafka resource upon catching an {@link Exception}
-     * raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}.
-     * After Kafka resource is destroyed it will be re-created upon the next
-     * invocation of this operation essentially providing a self healing
-     * mechanism to deal with potentially corrupted resource.
-     * <p>
-     * Keep in mind that upon catching an exception the state of this processor
-     * will be set to no longer accept any more tasks, until Kafka resource is
-     * reset. This means that in a multi-threaded situation currently executing
-     * tasks will be given a chance to complete while no new tasks will be
-     * accepted.
-     *
-     * @param context context
-     * @param sessionFactory factory
-     */
-    @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted.
-            this.taskCounter.incrementAndGet();
-            final ProcessSession session = sessionFactory.createSession();
-            try {
-                /*
-                 * We can't be doing double null check here since as a pattern
-                 * it only works for lazy init but not reset, which is what we
-                 * are doing here. In fact the first null check is dangerous
-                 * since 'kafkaPublisher' can become null right after its null
-                 * check passed causing subsequent NPE.
-                 */
-                synchronized (this) {
-                    if (this.kafkaPublisher == null) {
-                        this.kafkaPublisher = this.buildKafkaResource(context, session);
-                    }
-                }
-
-                /*
-                 * The 'processed' boolean flag does not imply any failure or success. It simply states that:
-                 * - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated
-                 * - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile
-                 */
-                boolean processed = this.rendezvousWithKafka(context, session);
-                session.commit();
-                if (!processed) {
-                    context.yield();
-                }
-            } catch (Throwable e) {
-                this.acceptTask = false;
-                session.rollback(true);
-                this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e});
-            } finally {
-                synchronized (this) {
-                    if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) {
-                        this.close();
-                        this.acceptTask = true;
-                    }
-                }
-            }
-        } else {
-            this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
-            this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
-            context.yield();
+    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 Map<String, Object> 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);
     }
 
-    /**
-     * Will call {@link Closeable#close()} on the target resource after which
-     * the target resource will be set to null. Should only be called when there
-     * are no more threads being executed on this processor or when it has been
-     * verified that only a single thread remains.
-     *
-     * @see KafkaPublisher
-     * @see KafkaConsumer
-     */
     @OnStopped
-    public void close() {
-        try {
-            if (this.kafkaPublisher != null) {
-                try {
-                    this.kafkaPublisher.close();
-                } catch (Exception e) {
-                    this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e);
-                }
-            }
-        } finally {
-            this.kafkaPublisher = null;
+    public void closePool() {
+        if (publisherPool != null) {
+            publisherPool.close();
         }
+
+        publisherPool = null;
     }
 
-    /**
-     * Will rendezvous with Kafka if {@link ProcessSession} contains
-     * {@link FlowFile} producing a result {@link FlowFile}.
-     * <br>
-     * The result {@link FlowFile} that is successful is then transfered to
-     * {@link #REL_SUCCESS}
-     * <br>
-     * The result {@link FlowFile} that is failed is then transfered to
-     * {@link #REL_FAILURE}
-     *
-     */
-    protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) {
-        FlowFile flowFile = session.get();
-        if (flowFile != null) {
-            long start = System.nanoTime();
-            flowFile = this.doRendezvousWithKafka(flowFile, context, session);
-            Relationship relationship = REL_SUCCESS;
-            if (!this.isFailedFlowFile(flowFile)) {
-                String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
-                long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-                String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic);
-                session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration);
-                this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis",
-                        new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration});
-            } else {
-                relationship = REL_FAILURE;
-                flowFile = session.penalize(flowFile);
-            }
-            session.transfer(flowFile, relationship);
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet();
+
+        final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500));
+        if (flowFiles.isEmpty()) {
+            return;
         }
-        return flowFile != null;
-    }
 
-    /**
-     * Builds and instance of {@link KafkaPublisher}.
-     */
-    protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) {
-        final Map<String, String> kafkaProps = new HashMap<>();
-        KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps);
-        kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
-        kafkaProps.put("max.request.size", String.valueOf(context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue()));
-        this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
-        final Properties props = new Properties();
-        props.putAll(kafkaProps);
-        KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger());
-        return publisher;
-    }
+        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 long startTime = System.nanoTime();
+        try (final PublisherLease lease = pool.obtainPublisher()) {
+            // Send each FlowFile to Kafka asynchronously.
+            for (final FlowFile flowFile : flowFiles) {
+                if (!isScheduled()) {
+                    // If stopped, re-queue FlowFile instead of sending it
+                    session.transfer(flowFile);
+                    continue;
+                }
 
-    /**
-     * Will rendezvous with {@link KafkaPublisher} after building
-     * {@link PublishingContext} and will produce the resulting
-     * {@link FlowFile}. The resulting FlowFile contains all required
-     * information to determine if message publishing originated from the
-     * provided FlowFile has actually succeeded fully, partially or failed
-     * completely (see {@link #isFailedFlowFile(FlowFile)}.
-     */
-    private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
-        final AtomicReference<KafkaPublisher.KafkaPublisherResult> publishResultRef = new AtomicReference<>();
-        session.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(InputStream contentStream) throws IOException {
-                PublishingContext publishingContext = PublishKafka_0_10.this.buildPublishingContext(flowFile, context, contentStream);
-                KafkaPublisher.KafkaPublisherResult result = PublishKafka_0_10.this.kafkaPublisher.publish(publishingContext);
-                publishResultRef.set(result);
+                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);
+                        }
+                    }
+                });
             }
-        });
 
-        FlowFile resultFile = publishResultRef.get().isAllAcked()
-                ? this.cleanUpFlowFileIfNecessary(flowFile, session)
-                : session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context));
+            // Complete the send
+            final PublishResult publishResult = lease.complete();
 
-        if (!this.isFailedFlowFile(resultFile)) {
-            resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent()));
-        }
-        return resultFile;
-    }
+            // Transfer any successful FlowFiles.
+            final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
+            for (FlowFile success : publishResult.getSuccessfulFlowFiles()) {
+                final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue();
 
-    /**
-     * Builds {@link PublishingContext} for message(s) to be sent to Kafka.
-     * {@link PublishingContext} contains all contextual information required by
-     * {@link KafkaPublisher} to publish to Kafka. Such information contains
-     * things like topic name, content stream, delimiter, key and last ACKed
-     * message for cases where provided FlowFile is being retried (failed in the
-     * past).
-     * <br>
-     * For the clean FlowFile (file that has been sent for the first time),
-     * PublishingContext will be built form {@link ProcessContext} associated
-     * with this invocation.
-     * <br>
-     * For the failed FlowFile, {@link PublishingContext} will be built from
-     * attributes of that FlowFile which by then will already contain required
-     * information (e.g., topic, key, delimiter etc.). This is required to
-     * ensure the affinity of the retry in the even where processor
-     * configuration has changed. However keep in mind that failed FlowFile is
-     * only considered a failed FlowFile if it is being re-processed by the same
-     * processor (determined via {@link #FAILED_PROC_ID_ATTR}, see
-     * {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to
-     * another PublishKafka0_10 processor it is treated as a fresh FlowFile
-     * regardless if it has #FAILED* attributes set.
-     */
-    private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) {
-        final byte[] keyBytes = getMessageKey(flowFile, context);
-
-        final String topicName;
-        final byte[] delimiterBytes;
-        int lastAckedMessageIndex = -1;
-        if (this.isFailedFlowFile(flowFile)) {
-            lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
-            topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
-            delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
-                    ? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
-        } else {
-            topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
-            delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR)
-                    .evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
-        }
+                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);
+            }
+
+            // Transfer any failures.
+            for (final FlowFile failure : publishResult.getFailedFlowFiles()) {
+                final int successCount = publishResult.getSuccessfulMessageCount(failure);
+                if (successCount > 0) {
+                    getLogger().error("Failed to send some messages for {} to Kafka, but {} messages were acknowledged by Kafka. Routing to failure due to {}",
+                        new Object[] {failure, successCount, publishResult.getReasonForFailure(failure)});
+                } else {
+                    getLogger().error("Failed to send all message for {} to Kafka; routing to failure due to {}",
+                        new Object[] {failure, publishResult.getReasonForFailure(failure)});
+                }
 
-        PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex,
-                context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue());
-        publishingContext.setKeyBytes(keyBytes);
-        publishingContext.setDelimiterBytes(delimiterBytes);
-        return publishingContext;
+                session.transfer(failure, REL_FAILURE);
+            }
+        }
     }
 
+
     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();
@@ -505,51 +394,4 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
 
         return DatatypeConverter.parseHexBinary(uninterpretedKey);
     }
-
-    /**
-     * Will remove FAILED_* attributes if FlowFile is no longer considered a
-     * failed FlowFile
-     *
-     * @see #isFailedFlowFile(FlowFile)
-     */
-    private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) {
-        if (this.isFailedFlowFile(flowFile)) {
-            Set<String> keysToRemove = new HashSet<>();
-            keysToRemove.add(FAILED_DELIMITER_ATTR);
-            keysToRemove.add(FAILED_KEY_ATTR);
-            keysToRemove.add(FAILED_TOPIC_ATTR);
-            keysToRemove.add(FAILED_PROC_ID_ATTR);
-            keysToRemove.add(FAILED_LAST_ACK_IDX);
-            flowFile = session.removeAllAttributes(flowFile, keysToRemove);
-        }
-        return flowFile;
-    }
-
-    /**
-     * Builds a {@link Map} of FAILED_* attributes
-     *
-     * @see #FAILED_PROC_ID_ATTR
-     * @see #FAILED_LAST_ACK_IDX
-     * @see #FAILED_TOPIC_ATTR
-     * @see #FAILED_KEY_ATTR
-     * @see #FAILED_DELIMITER_ATTR
-     */
-    private Map<String, String> buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile, ProcessContext context) {
-        Map<String, String> attributes = new HashMap<>();
-        attributes.put(FAILED_PROC_ID_ATTR, this.getIdentifier());
-        attributes.put(FAILED_LAST_ACK_IDX, String.valueOf(lastAckedMessageIndex));
-        attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue());
-        attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue());
-        attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet()
-                ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null);
-        return attributes;
-    }
-
-    /**
-     * Returns 'true' if provided FlowFile is a failed FlowFile. A failed
-     * FlowFile contains {@link #FAILED_PROC_ID_ATTR}.
-     */
-    private boolean isFailedFlowFile(FlowFile flowFile) {
-        return this.getIdentifier().equals(flowFile.getAttribute(FAILED_PROC_ID_ATTR));
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
new file mode 100644
index 0000000..b685265
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishResult.java
@@ -0,0 +1,56 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+
+import org.apache.nifi.flowfile.FlowFile;
+
+public interface PublishResult {
+    Collection<FlowFile> getSuccessfulFlowFiles();
+
+    Collection<FlowFile> getFailedFlowFiles();
+
+    int getSuccessfulMessageCount(FlowFile flowFile);
+
+    Exception getReasonForFailure(FlowFile flowFile);
+
+
+    public static final PublishResult EMPTY = new PublishResult() {
+        @Override
+        public Collection<FlowFile> getSuccessfulFlowFiles() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public Collection<FlowFile> getFailedFlowFiles() {
+            return Collections.emptyList();
+        }
+
+        @Override
+        public int getSuccessfulMessageCount(FlowFile flowFile) {
+            return 0;
+        }
+
+        @Override
+        public Exception getReasonForFailure(FlowFile flowFile) {
+            return null;
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
new file mode 100644
index 0000000..b67e8a8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+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.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+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<byte[], byte[]> producer;
+    private final int maxMessageSize;
+    private final long maxAckWaitMillis;
+    private volatile boolean poisoned = false;
+
+    private InFlightMessageTracker tracker;
+
+    public PublisherLease(final Producer<byte[], byte[]> producer, final int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
+        this.producer = producer;
+        this.maxMessageSize = maxMessageSize;
+        this.logger = logger;
+        this.maxAckWaitMillis = maxAckWaitMillis;
+    }
+
+    protected void poison() {
+        this.poisoned = true;
+    }
+
+    public boolean isPoisoned() {
+        return poisoned;
+    }
+
+    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();
+        }
+
+        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;
+        }
+    }
+
+    private void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker) {
+        final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, null, messageKey, messageContent);
+        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();
+                }
+            }
+        });
+
+        tracker.incrementSentCount(flowFile);
+    }
+
+    public PublishResult complete() {
+        if (tracker == null) {
+            throw new IllegalStateException("Cannot complete publishing to Kafka because Publisher Lease was already closed");
+        }
+
+        producer.flush();
+
+        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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
new file mode 100644
index 0000000..5902b03
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import java.io.Closeable;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+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<PublisherLease> publisherQueue;
+    private final Map<String, Object> kafkaProperties;
+    private final int maxMessageSize;
+    private final long maxAckWaitMillis;
+
+    private volatile boolean closed = false;
+
+    PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis) {
+        this.logger = logger;
+        this.publisherQueue = new LinkedBlockingQueue<>();
+        this.kafkaProperties = kafkaProperties;
+        this.maxMessageSize = maxMessageSize;
+        this.maxAckWaitMillis = maxAckWaitMillis;
+    }
+
+    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 Producer<byte[], byte[]> producer = new KafkaProducer<>(kafkaProperties);
+        final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger) {
+            @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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
deleted file mode 100644
index 1513481..0000000
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishingContext.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.kafka.pubsub;
-
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-
-/**
- * Holder of context information used by {@link KafkaPublisher} required to
- * publish messages to Kafka.
- */
-class PublishingContext {
-
-    private final InputStream contentStream;
-
-    private final String topic;
-
-    private final int lastAckedMessageIndex;
-
-    private final int maxRequestSize;
-
-    private byte[] keyBytes;
-
-    private byte[] delimiterBytes;
-
-    PublishingContext(InputStream contentStream, String topic) {
-        this(contentStream, topic, -1);
-    }
-
-    PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
-        this(contentStream, topic, lastAckedMessageIndex, 1048576);
-    }
-
-    PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex, int maxRequestSize) {
-        this.validateInput(contentStream, topic, lastAckedMessageIndex);
-        this.contentStream = contentStream;
-        this.topic = topic;
-        this.lastAckedMessageIndex = lastAckedMessageIndex;
-        this.maxRequestSize = maxRequestSize;
-    }
-
-    @Override
-    public String toString() {
-        return "topic: '" + this.topic + "'; delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'";
-    }
-
-    int getLastAckedMessageIndex() {
-        return this.lastAckedMessageIndex;
-    }
-
-    int getMaxRequestSize() {
-        return this.maxRequestSize;
-    }
-
-    byte[] getKeyBytes() {
-        return this.keyBytes;
-    }
-
-    byte[] getDelimiterBytes() {
-        return this.delimiterBytes;
-    }
-
-    InputStream getContentStream() {
-        return this.contentStream;
-    }
-
-    String getTopic() {
-        return this.topic;
-    }
-
-    void setKeyBytes(byte[] keyBytes) {
-        if (this.keyBytes == null) {
-            if (keyBytes != null) {
-                this.assertBytesValid(keyBytes);
-                this.keyBytes = keyBytes;
-            }
-        } else {
-            throw new IllegalArgumentException("'keyBytes' can only be set once per instance");
-        }
-    }
-
-    void setDelimiterBytes(byte[] delimiterBytes) {
-        if (this.delimiterBytes == null) {
-            if (delimiterBytes != null) {
-                this.assertBytesValid(delimiterBytes);
-                this.delimiterBytes = delimiterBytes;
-            }
-        } else {
-            throw new IllegalArgumentException("'delimiterBytes' can only be set once per instance");
-        }
-    }
-
-    private void assertBytesValid(byte[] bytes) {
-        if (bytes != null) {
-            if (bytes.length == 0) {
-                throw new IllegalArgumentException("'bytes' must not be empty");
-            }
-        }
-    }
-
-    private void validateInput(InputStream contentStream, String topic, int lastAckedMessageIndex) {
-        if (contentStream == null) {
-            throw new IllegalArgumentException("'contentStream' must not be null");
-        } else if (topic == null || topic.trim().length() == 0) {
-            throw new IllegalArgumentException("'topic' must not be null or empty");
-        } else if (lastAckedMessageIndex < -1) {
-            throw new IllegalArgumentException("'lastAckedMessageIndex' must be >= -1");
-        }
-    }
-}


[17/20] nifi git commit: [NIFI-2781] removing bower and allowing npm to manage client side resources in nifi-web-ui and nifi-jolt-transform-json-ui. This closes #1030

Posted by jp...@apache.org.
[NIFI-2781] removing bower and allowing npm to manage client side resources in nifi-web-ui and nifi-jolt-transform-json-ui. This closes #1030


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

Branch: refs/heads/support/nifi-1.0.x
Commit: af1732aa610e5b409b295dc35be17d4ab45c7375
Parents: 0b1d15a
Author: Scott Aslan <sc...@gmail.com>
Authored: Mon Sep 19 13:16:07 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Thu Dec 15 09:52:12 2016 -0500

----------------------------------------------------------------------
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml | 39 +++++++++++-------
 .../nifi-web-ui/src/main/frontend/bower.json    | 20 ----------
 .../nifi-web-ui/src/main/frontend/package.json  | 14 +++++++
 .../nifi-jolt-transform-json-ui/pom.xml         | 42 +++++++++++---------
 .../src/main/frontend/.bowerrc                  |  6 ---
 .../src/main/frontend/bower.json                | 15 -------
 .../src/main/frontend/package.json              | 12 ++++++
 .../src/main/webapp/WEB-INF/jsp/index.jsp       |  2 +-
 8 files changed, 75 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index a8c1994..c96076e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -136,7 +136,26 @@
                         </configuration>
                     </execution>
                     <execution>
-                        <id>copy-bower-json</id>
+                        <id>copy-client-side-deps</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>copy-resources</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${frontend.assets}</outputDirectory>
+                            <resources>
+                                <resource>
+                                    <directory>${frontend.working.dir}/node_modules</directory>
+                                    <filtering>false</filtering>
+                                    <includes>
+                                        <include>**/*</include>
+                                    </includes>
+                                </resource>
+                            </resources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>copy-package-json</id>
                         <phase>generate-sources</phase>
                         <goals>
                             <goal>copy-resources</goal>
@@ -148,7 +167,7 @@
                                     <directory>${frontend.dependency.configs}</directory>
                                     <filtering>false</filtering>
                                     <includes>
-                                        <include>bower.json</include>
+                                        <include>package.json</include>
                                     </includes>
                                 </resource>
                             </resources>
@@ -271,22 +290,12 @@
                         </configuration>
                     </execution>
                     <execution>
-                        <id>install-bower</id>
+                        <id>npm install</id>
                         <goals>
                             <goal>npm</goal>
                         </goals>
                         <configuration>
-                            <arguments>install bower</arguments>
-                            <workingDirectory>${frontend.working.dir}</workingDirectory>
-                        </configuration>
-                    </execution>
-                    <execution>
-                        <id>bower-install</id>
-                        <goals>
-                            <goal>bower</goal>
-                        </goals>
-                        <configuration>
-                            <arguments>install ${bower.options} --config.directory=${frontend.assets}</arguments>
+                            <arguments>--cache-min Infinity install</arguments>
                             <workingDirectory>${frontend.working.dir}</workingDirectory>
                         </configuration>
                     </execution>
@@ -787,7 +796,7 @@
                         <configuration>
                             <excludes combine.children="append">
                                 <exclude>nbactions.xml</exclude>
-                                <exclude>src/main/frontend/bower.json</exclude>
+                                <exclude>src/main/frontend/package.json</exclude>
                                 <exclude>src/main/webapp/js/json2.js</exclude>
                                 <exclude>src/main/webapp/js/jquery/</exclude>
                                 <exclude>src/main/webapp/js/d3/d3.min.js</exclude>

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/bower.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/bower.json b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/bower.json
deleted file mode 100644
index 2544686..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/bower.json
+++ /dev/null
@@ -1,20 +0,0 @@
-{
-  "name": "apache-nifi",
-  "private": true,
-  "ignore": [
-    "**/.*",
-    "node_modules",
-    "bower_components",
-    "test",
-    "tests"
-  ],
-  "dependencies": {
-    "font-awesome": "fontawesome#4.6.1",
-    "angular-material": "1.0.8",
-    "angular-resource": "1.5.5",
-    "angular-route": "1.5.5"
-  },
-  "resolutions": {
-    "angular": "1.5.8"
-  }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json
new file mode 100644
index 0000000..67acdb1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json
@@ -0,0 +1,14 @@
+{
+  "name": "apache-nifi",
+  "dependencies": {
+    "font-awesome": "4.6.1",
+    "angular-material": "1.0.8",
+    "angular-resource": "1.5.5",
+    "angular-route": "1.5.5"
+  },
+  "description": "Apache NiFi 3rd party client side resources.",
+  "repository": {
+    "type": "git",
+    "url": "https://github.com/apache/nifi"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/pom.xml
index 4607cfd..209c06b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/pom.xml
@@ -144,7 +144,26 @@ language governing permissions and limitations under the License. -->
                 <artifactId>maven-resources-plugin</artifactId>
                 <executions>
                     <execution>
-                        <id>copy-bower-json</id>
+                        <id>copy-client-side-deps</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>copy-resources</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${frontend.assets}</outputDirectory>
+                            <resources>
+                                <resource>
+                                    <directory>${frontend.working.dir}/node_modules</directory>
+                                    <filtering>false</filtering>
+                                    <includes>
+                                        <include>**/*</include>
+                                    </includes>
+                                </resource>
+                            </resources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>copy-package-json</id>
                         <phase>generate-sources</phase>
                         <goals>
                             <goal>copy-resources</goal>
@@ -156,8 +175,7 @@ language governing permissions and limitations under the License. -->
                                     <directory>${frontend.dependency.configs}</directory>
                                     <filtering>false</filtering>
                                     <includes>
-                                        <include>bower.json</include>
-                                        <include>.bowerrc</include>
+                                        <include>package.json</include>
                                     </includes>
                                 </resource>
                             </resources>
@@ -172,7 +190,6 @@ language governing permissions and limitations under the License. -->
                 <configuration>
                     <installDirectory>${frontend.working.dir}</installDirectory>
                 </configuration>
-
                 <executions>
                     <execution>
                         <id>install-node-and-npm</id>
@@ -186,22 +203,12 @@ language governing permissions and limitations under the License. -->
                         </configuration>
                     </execution>
                     <execution>
-                        <id>install-bower</id>
+                        <id>npm install</id>
                         <goals>
                             <goal>npm</goal>
                         </goals>
                         <configuration>
-                            <arguments>install bower</arguments>
-                            <workingDirectory>${frontend.working.dir}</workingDirectory>
-                        </configuration>
-                    </execution>
-                    <execution>
-                        <id>bower-install</id>
-                        <goals>
-                            <goal>bower</goal>
-                        </goals>
-                        <configuration>
-                            <arguments>install --config.directory=${frontend.assets}</arguments>
+                            <arguments>--cache-min Infinity install</arguments>
                             <workingDirectory>${frontend.working.dir}</workingDirectory>
                         </configuration>
                     </execution>
@@ -212,8 +219,7 @@ language governing permissions and limitations under the License. -->
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes combine.children="append">
-                        <exclude>src/main/frontend/.bowerrc</exclude>
-                        <exclude>src/main/frontend/bower.json</exclude>
+                        <exclude>src/main/frontend/package.json</exclude>
                         <exclude>src/main/webapp/css/main.css</exclude>
                         <exclude>src/main/webapp/js/jsonlint/*</exclude>
                         <exclude>src/main/webapp/js/js-beautify/*</exclude>

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/.bowerrc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/.bowerrc b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/.bowerrc
deleted file mode 100644
index 31190ed..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/.bowerrc
+++ /dev/null
@@ -1,6 +0,0 @@
-{
-  "ignoredDependencies": [
-    "angular",
-    "codemirror"
-  ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/bower.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/bower.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/bower.json
deleted file mode 100644
index 08f08c6..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/bower.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
-  "name": "apache-nifi",
-  "private": true,
-  "ignore": [
-    "**/.*",
-    "node_modules",
-    "bower_components",
-    "test",
-    "tests"
-  ],
-  "dependencies": {
-    "angular-ui-codemirror": "^0.3.0",
-    "angular-ui-router": "^0.2.18"
-  }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/package.json
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/package.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/package.json
new file mode 100644
index 0000000..2899812
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/frontend/package.json
@@ -0,0 +1,12 @@
+{
+  "name": "nifi-jolt-transform-json-ui",
+  "dependencies": {
+    "angular-ui-codemirror": "^0.3.0",
+    "angular-ui-router": "^0.2.18"
+  },
+  "description": "Apache NiFi Jolt Transform JSON UI 3rd party client side resources.",
+  "repository": {
+    "type": "git",
+    "url": "https://github.com/apache/nifi"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/af1732aa/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
index 750bd8b..96eda99 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
@@ -38,7 +38,7 @@
 <script type="text/javascript" src="../nifi/assets/angular-material/angular-material.min.js"></script>
 
 <!--Bower Libraries-->
-<script type="text/javascript" src="assets/angular-ui-codemirror/ui-codemirror.min.js"></script>
+<script type="text/javascript" src="assets/angular-ui-codemirror/src/ui-codemirror.js"></script>
 <script type="text/javascript" src="assets/angular-ui-router/release/angular-ui-router.min.js"></script>
 
 <!--Local Libraries-->


[19/20] nifi git commit: NIFI-2822 - pass the JWT Authorization token i. This closes #1065

Posted by jp...@apache.org.
NIFI-2822 - pass the JWT Authorization token i. This closes #1065


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

Branch: refs/heads/support/nifi-1.0.x
Commit: e80a670b0ecc9b6a06e415010c188793b5575021
Parents: cdfed07
Author: Yolanda M. Davis <yo...@gmail.com>
Authored: Mon Sep 26 12:11:28 2016 -0400
Committer: jpercivall <JP...@apache.org>
Committed: Thu Dec 15 11:16:59 2016 -0500

----------------------------------------------------------------------
 .../src/main/webapp/WEB-INF/jsp/index.jsp                   | 2 ++
 .../nifi-jolt-transform-json-ui/src/main/webapp/app/app.js  | 9 +++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/e80a670b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
index 96eda99..26ba321 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/WEB-INF/jsp/index.jsp
@@ -31,6 +31,8 @@
 <script type="text/javascript" src="../nifi/js/codemirror/addon/lint/lint.js"></script>
 <script type="text/javascript" src="../nifi/js/codemirror/addon/lint/json-lint.js"></script>
 <script type="text/javascript" src="../nifi/js/jsonlint/jsonlint.min.js"></script>
+<script type="text/javascript" src="../nifi/js/nf/nf-namespace.js"></script>
+<script type="text/javascript" src="../nifi/js/nf/nf-storage.js"></script>
 <script type="text/javascript" src="../nifi/assets/angular/angular.min.js"></script>
 <script type="text/javascript" src="../nifi/assets/angular-animate/angular-animate.min.js"></script>
 <script type="text/javascript" src="../nifi/assets/angular-aria/angular-aria.min.js"></script>

http://git-wip-us.apache.org/repos/asf/nifi/blob/e80a670b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/app/app.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/app/app.js b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/app/app.js
index 67c440c..2e9cf0c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/app/app.js
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-jolt-transform-json-ui/src/main/webapp/app/app.js
@@ -17,7 +17,12 @@
 
 'use strict';
 
-var AppRun =  function($rootScope,$state){
+var AppRun =  function($rootScope,$state,$http){
+
+    if (nf.Storage.hasItem('jwt')) {
+        var token = nf.Storage.getItem('jwt');
+        $http.defaults.headers.common.Authorization = 'Bearer ' + token;
+    }
 
     $rootScope.$on('$stateChangeError', function(event, toState, toParams, fromState, fromParams, error){
         event.preventDefault();
@@ -35,7 +40,7 @@ var AppConfig = function ($urlRouterProvider) {
 
 };
 
-AppRun.$inject = ['$rootScope','$state'];
+AppRun.$inject = ['$rootScope','$state','$http'];
 
 AppConfig.$inject = ['$urlRouterProvider'];
 


[12/20] nifi git commit: NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.

Posted by jp...@apache.org.
NIFI-3198: Refactored how PublishKafka and PublishKafka_0_10 work to improve throughput and resilience. Fixed bug in StreamDemarcator. Slight refactoring of consume processors to simplify code.


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

Branch: refs/heads/support/nifi-1.0.x
Commit: ed17df503b6d842a439e549615c2750df6752632
Parents: 1ba7f83
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Dec 14 14:23:21 2016 -0500
Committer: jpercivall <JP...@apache.org>
Committed: Wed Dec 14 16:23:50 2016 -0500

----------------------------------------------------------------------
 .../io/exception/TokenTooLargeException.java    |  26 +
 .../nifi/stream/io/util/StreamDemarcator.java   |  39 +-
 .../stream/io/util/StreamDemarcatorTest.java    |  84 ++-
 .../nifi/util/StandardProcessorTestRunner.java  |  31 +-
 .../java/org/apache/nifi/util/TestRunner.java   |  16 +-
 .../kafka/pubsub/ConsumeKafka_0_10.java         | 315 +++------
 .../processors/kafka/pubsub/ConsumerLease.java  | 367 ++++++++++-
 .../processors/kafka/pubsub/ConsumerPool.java   | 287 ++++----
 .../kafka/pubsub/InFlightMessageTracker.java    | 178 +++++
 .../kafka/pubsub/KafkaProcessorUtils.java       |  37 +-
 .../processors/kafka/pubsub/KafkaPublisher.java | 236 -------
 .../kafka/pubsub/PublishKafka_0_10.java         | 646 +++++++------------
 .../processors/kafka/pubsub/PublishResult.java  |  56 ++
 .../processors/kafka/pubsub/PublisherLease.java | 132 ++++
 .../processors/kafka/pubsub/PublisherPool.java  |  98 +++
 .../kafka/pubsub/PublishingContext.java         | 124 ----
 .../kafka/pubsub/ConsumeKafkaTest.java          | 548 ++--------------
 .../kafka/pubsub/ConsumerPoolTest.java          | 172 +++--
 .../kafka/pubsub/KafkaPublisherTest.java        | 306 ---------
 .../kafka/pubsub/PublishKafkaTest.java          | 375 -----------
 .../kafka/pubsub/PublishingContextTest.java     |  91 ---
 .../kafka/pubsub/StubPublishKafka.java          | 143 ----
 .../pubsub/TestInFlightMessageTracker.java      |  87 +++
 .../kafka/pubsub/TestPublishKafka.java          | 262 ++++++++
 .../kafka/pubsub/TestPublisherLease.java        | 194 ++++++
 .../kafka/pubsub/TestPublisherPool.java         |  68 ++
 .../nifi/processors/kafka/KafkaPublisher.java   |   4 +-
 .../processors/kafka/pubsub/ConsumeKafka.java   | 313 +++------
 .../processors/kafka/pubsub/ConsumerLease.java  | 367 ++++++++++-
 .../processors/kafka/pubsub/ConsumerPool.java   | 288 ++++-----
 .../kafka/pubsub/InFlightMessageTracker.java    | 178 +++++
 .../kafka/pubsub/KafkaProcessorUtils.java       |  37 +-
 .../processors/kafka/pubsub/KafkaPublisher.java | 236 -------
 .../processors/kafka/pubsub/PublishKafka.java   | 641 +++++++-----------
 .../processors/kafka/pubsub/PublishResult.java  |  56 ++
 .../processors/kafka/pubsub/PublisherLease.java | 132 ++++
 .../processors/kafka/pubsub/PublisherPool.java  |  98 +++
 .../kafka/pubsub/PublishingContext.java         | 124 ----
 .../kafka/pubsub/ConsumeKafkaTest.java          | 555 ++--------------
 .../kafka/pubsub/ConsumerPoolTest.java          | 172 +++--
 .../kafka/pubsub/KafkaPublisherTest.java        | 306 ---------
 .../kafka/pubsub/PublishKafkaTest.java          | 375 -----------
 .../kafka/pubsub/PublishingContextTest.java     |  91 ---
 .../kafka/pubsub/StubPublishKafka.java          | 144 -----
 .../pubsub/TestInFlightMessageTracker.java      |  87 +++
 .../kafka/pubsub/TestPublishKafka.java          | 262 ++++++++
 .../kafka/pubsub/TestPublisherLease.java        | 194 ++++++
 .../kafka/pubsub/TestPublisherPool.java         |  68 ++
 48 files changed, 4338 insertions(+), 5308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/TokenTooLargeException.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/TokenTooLargeException.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/TokenTooLargeException.java
new file mode 100644
index 0000000..7024f34
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/exception/TokenTooLargeException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.stream.io.exception;
+
+import java.io.IOException;
+
+public class TokenTooLargeException extends IOException {
+    public TokenTooLargeException(final String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/StreamDemarcator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/StreamDemarcator.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/StreamDemarcator.java
index 3064f1c..840bdb0 100644
--- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/StreamDemarcator.java
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/util/StreamDemarcator.java
@@ -16,9 +16,12 @@
  */
 package org.apache.nifi.stream.io.util;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStream;
 
+import org.apache.nifi.stream.io.exception.TokenTooLargeException;
+
 /**
  * The <code>StreamDemarcator</code> class takes an input stream and demarcates
  * it so it could be read (see {@link #nextToken()}) as individual byte[]
@@ -26,7 +29,7 @@ import java.io.InputStream;
  * stream will be read into a single token which may result in
  * {@link OutOfMemoryError} if stream is too large.
  */
-public class StreamDemarcator {
+public class StreamDemarcator implements Closeable {
 
     private final static int INIT_BUFFER_SIZE = 8192;
 
@@ -95,8 +98,10 @@ public class StreamDemarcator {
     /**
      * Will read the next data token from the {@link InputStream} returning null
      * when it reaches the end of the stream.
+     *
+     * @throws IOException if unable to read from the stream
      */
-    public byte[] nextToken() {
+    public byte[] nextToken() throws IOException {
         byte[] data = null;
         int j = 0;
 
@@ -126,8 +131,10 @@ public class StreamDemarcator {
     /**
      * Will fill the current buffer from current 'index' position, expanding it
      * and or shuffling it if necessary
+     *
+     * @throws IOException if unable to read from the stream
      */
-    private void fill() {
+    private void fill() throws IOException {
         if (this.index >= this.buffer.length) {
             if (this.mark == 0) { // expand
                 byte[] newBuff = new byte[this.buffer.length + this.initialBufferSize];
@@ -138,23 +145,20 @@ public class StreamDemarcator {
                 System.arraycopy(this.buffer, this.mark, this.buffer, 0, length);
                 this.index = length;
                 this.mark = 0;
+                this.readAheadLength = length;
             }
         }
 
-        try {
-            int bytesRead;
-            do {
-                bytesRead = this.is.read(this.buffer, this.index, this.buffer.length - this.index);
-            } while (bytesRead == 0);
+        int bytesRead;
+        do {
+            bytesRead = this.is.read(this.buffer, this.index, this.buffer.length - this.index);
+        } while (bytesRead == 0);
 
-            if (bytesRead != -1) {
-                this.readAheadLength = this.index + bytesRead;
-                if (this.readAheadLength > this.maxDataSize) {
-                    throw new IllegalStateException("Maximum allowed data size of " + this.maxDataSize + " exceeded.");
-                }
+        if (bytesRead != -1) {
+            this.readAheadLength = this.index + bytesRead;
+            if (this.readAheadLength > this.maxDataSize) {
+                throw new TokenTooLargeException("A message in the stream exceeds the maximum allowed message size of " + this.maxDataSize + " bytes.");
             }
-        } catch (IOException e) {
-            throw new IllegalStateException("Failed while reading InputStream", e);
         }
     }
 
@@ -188,4 +192,9 @@ public class StreamDemarcator {
             throw new IllegalArgumentException("'delimiterBytes' is an optional argument, but when provided its length must be > 0");
         }
     }
+
+    @Override
+    public void close() throws IOException {
+        is.close();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/util/StreamDemarcatorTest.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/util/StreamDemarcatorTest.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/util/StreamDemarcatorTest.java
index 93082a2..66d2668 100644
--- a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/util/StreamDemarcatorTest.java
+++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/util/StreamDemarcatorTest.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.ByteArrayInputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -65,7 +66,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateNoDelimiter() {
+    public void validateNoDelimiter() throws IOException {
         String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, null, 1000);
@@ -76,7 +77,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateNoDelimiterSmallInitialBuffer() {
+    public void validateNoDelimiterSmallInitialBuffer() throws IOException {
         String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, null, 1000, 1);
@@ -84,7 +85,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateSingleByteDelimiter() {
+    public void validateSingleByteDelimiter() throws IOException {
         String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@@ -95,7 +96,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateDelimiterAtTheBeginning() {
+    public void validateDelimiterAtTheBeginning() throws IOException {
         String data = ",Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@@ -106,7 +107,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateEmptyDelimiterSegments() {
+    public void validateEmptyDelimiterSegments() throws IOException {
         String data = ",,,,,Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@@ -117,7 +118,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateSingleByteDelimiterSmallInitialBuffer() {
+    public void validateSingleByteDelimiterSmallInitialBuffer() throws IOException {
         String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000, 2);
@@ -128,7 +129,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithMultiByteDelimiter() {
+    public void validateWithMultiByteDelimiter() throws IOException {
         String data = "foodaabardaabazzz";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
@@ -139,7 +140,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithMultiByteDelimiterAtTheBeginning() {
+    public void validateWithMultiByteDelimiterAtTheBeginning() throws IOException {
         String data = "daafoodaabardaabazzz";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
@@ -150,7 +151,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithMultiByteDelimiterSmallInitialBuffer() {
+    public void validateWithMultiByteDelimiterSmallInitialBuffer() throws IOException {
         String data = "foodaabarffdaabazz";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000, 1);
@@ -161,7 +162,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithMultiByteCharsNoDelimiter() {
+    public void validateWithMultiByteCharsNoDelimiter() throws IOException {
         String data = "\u50e0THIS IS MY NEW TEXT.\u50e0IT HAS A NEWLINE.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, null, 1000);
@@ -172,7 +173,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithMultiByteCharsNoDelimiterSmallInitialBuffer() {
+    public void validateWithMultiByteCharsNoDelimiterSmallInitialBuffer() throws IOException {
         String data = "\u50e0THIS IS MY NEW TEXT.\u50e0IT HAS A NEWLINE.";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
         StreamDemarcator scanner = new StreamDemarcator(is, null, 1000, 2);
@@ -183,7 +184,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateWithComplexDelimiter() {
+    public void validateWithComplexDelimiter() throws IOException {
         String data = "THIS IS MY TEXT<MYDELIMITER>THIS IS MY NEW TEXT<MYDELIMITER>THIS IS MY NEWEST TEXT";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
         StreamDemarcator scanner = new StreamDemarcator(is, "<MYDELIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
@@ -193,8 +194,8 @@ public class StreamDemarcatorTest {
         assertNull(scanner.nextToken());
     }
 
-    @Test(expected = IllegalStateException.class)
-    public void validateMaxBufferSize() {
+    @Test(expected = IOException.class)
+    public void validateMaxBufferSize() throws IOException {
         String data = "THIS IS MY TEXT<MY DELIMITER>THIS IS MY NEW TEXT<MY DELIMITER>THIS IS MY NEWEST TEXT";
         ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
         StreamDemarcator scanner = new StreamDemarcator(is, "<MY DELIMITER>".getBytes(StandardCharsets.UTF_8), 20);
@@ -202,7 +203,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateScannerHandlesNegativeOneByteInputsNoDelimiter() {
+    public void validateScannerHandlesNegativeOneByteInputsNoDelimiter() throws IOException {
         ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
         StreamDemarcator scanner = new StreamDemarcator(is, null, 20);
         byte[] b = scanner.nextToken();
@@ -210,7 +211,7 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void validateScannerHandlesNegativeOneByteInputs() {
+    public void validateScannerHandlesNegativeOneByteInputs() throws IOException {
         ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
         StreamDemarcator scanner = new StreamDemarcator(is, "water".getBytes(StandardCharsets.UTF_8), 20, 1024);
         byte[] b = scanner.nextToken();
@@ -218,10 +219,59 @@ public class StreamDemarcatorTest {
     }
 
     @Test
-    public void verifyScannerHandlesNegativeOneByteDelimiter() {
+    public void verifyScannerHandlesNegativeOneByteDelimiter() throws IOException {
         ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
         StreamDemarcator scanner = new StreamDemarcator(is, new byte[] { -1 }, 20, 1024);
         assertArrayEquals(scanner.nextToken(), new byte[] { 0, 0, 0, 0 });
         assertArrayEquals(scanner.nextToken(), new byte[] { 0, 0, 0 });
     }
+
+    @Test
+    public void testWithoutTrailingDelimiter() throws IOException {
+        final byte[] inputData = "Larger Message First\nSmall".getBytes(StandardCharsets.UTF_8);
+        ByteArrayInputStream is = new ByteArrayInputStream(inputData);
+        StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000);
+
+        final byte[] first = scanner.nextToken();
+        final byte[] second = scanner.nextToken();
+        assertNotNull(first);
+        assertNotNull(second);
+
+        assertEquals("Larger Message First", new String(first, StandardCharsets.UTF_8));
+        assertEquals("Small", new String(second, StandardCharsets.UTF_8));
+    }
+
+    @Test
+    public void testOnBufferSplitNoTrailingDelimiter() throws IOException {
+        final byte[] inputData = "Yes\nNo".getBytes(StandardCharsets.UTF_8);
+        ByteArrayInputStream is = new ByteArrayInputStream(inputData);
+        StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000, 3);
+
+        final byte[] first = scanner.nextToken();
+        final byte[] second = scanner.nextToken();
+        assertNotNull(first);
+        assertNotNull(second);
+
+        assertArrayEquals(first, new byte[] {'Y', 'e', 's'});
+        assertArrayEquals(second, new byte[] {'N', 'o'});
+    }
+
+    @Test
+    public void testOnBufferSplit() throws IOException {
+        final byte[] inputData = "123\n456\n789".getBytes(StandardCharsets.UTF_8);
+        ByteArrayInputStream is = new ByteArrayInputStream(inputData);
+        StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000, 3);
+
+        final byte[] first = scanner.nextToken();
+        final byte[] second = scanner.nextToken();
+        final byte[] third = scanner.nextToken();
+        assertNotNull(first);
+        assertNotNull(second);
+        assertNotNull(third);
+
+        assertArrayEquals(first, new byte[] {'1', '2', '3'});
+        assertArrayEquals(second, new byte[] {'4', '5', '6'});
+        assertArrayEquals(third, new byte[] {'7', '8', '9'});
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 69118db..e1de2b9 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -368,54 +368,55 @@ public class StandardProcessorTestRunner implements TestRunner {
     }
 
     @Override
-    public void enqueue(final Path path) throws IOException {
-        enqueue(path, new HashMap<String, String>());
+    public MockFlowFile enqueue(final Path path) throws IOException {
+        return enqueue(path, new HashMap<String, String>());
     }
 
     @Override
-    public void enqueue(final Path path, final Map<String, String> attributes) throws IOException {
+    public MockFlowFile enqueue(final Path path, final Map<String, String> attributes) throws IOException {
         final Map<String, String> modifiedAttributes = new HashMap<>(attributes);
         if (!modifiedAttributes.containsKey(CoreAttributes.FILENAME.key())) {
             modifiedAttributes.put(CoreAttributes.FILENAME.key(), path.toFile().getName());
         }
         try (final InputStream in = Files.newInputStream(path)) {
-            enqueue(in, modifiedAttributes);
+            return enqueue(in, modifiedAttributes);
         }
     }
 
     @Override
-    public void enqueue(final byte[] data) {
-        enqueue(data, new HashMap<String, String>());
+    public MockFlowFile enqueue(final byte[] data) {
+        return enqueue(data, new HashMap<String, String>());
     }
 
     @Override
-    public void enqueue(final String data) {
-        enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.<String, String> emptyMap());
+    public MockFlowFile enqueue(final String data) {
+        return enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.<String, String> emptyMap());
     }
 
     @Override
-    public void enqueue(final byte[] data, final Map<String, String> attributes) {
-        enqueue(new ByteArrayInputStream(data), attributes);
+    public MockFlowFile enqueue(final byte[] data, final Map<String, String> attributes) {
+        return enqueue(new ByteArrayInputStream(data), attributes);
     }
 
     @Override
-    public void enqueue(final String data, final Map<String, String> attributes) {
-        enqueue(data.getBytes(StandardCharsets.UTF_8), attributes);
+    public MockFlowFile enqueue(final String data, final Map<String, String> attributes) {
+        return enqueue(data.getBytes(StandardCharsets.UTF_8), attributes);
     }
 
 
     @Override
-    public void enqueue(final InputStream data) {
-        enqueue(data, new HashMap<String, String>());
+    public MockFlowFile enqueue(final InputStream data) {
+        return enqueue(data, new HashMap<String, String>());
     }
 
     @Override
-    public void enqueue(final InputStream data, final Map<String, String> attributes) {
+    public MockFlowFile enqueue(final InputStream data, final Map<String, String> attributes) {
         final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor);
         MockFlowFile flowFile = session.create();
         flowFile = session.importFrom(data, flowFile);
         flowFile = session.putAllAttributes(flowFile, attributes);
         enqueue(flowFile);
+        return flowFile;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
index 78d4d00..5d8b494 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
@@ -382,7 +382,7 @@ public interface TestRunner {
      * @param path to read content from
      * @throws IOException if unable to read content
      */
-    void enqueue(Path path) throws IOException;
+    MockFlowFile enqueue(Path path) throws IOException;
 
     /**
      * Reads the content from the given {@link Path} into memory and creates a
@@ -393,7 +393,7 @@ public interface TestRunner {
      * @param attributes attributes to use for new flow file
      * @throws IOException if unable to read content
      */
-    void enqueue(Path path, Map<String, String> attributes) throws IOException;
+    MockFlowFile enqueue(Path path, Map<String, String> attributes) throws IOException;
 
     /**
      * Copies the content from the given byte array into memory and creates a
@@ -402,7 +402,7 @@ public interface TestRunner {
      *
      * @param data to enqueue
      */
-    void enqueue(byte[] data);
+    MockFlowFile enqueue(byte[] data);
 
     /**
      * Creates a FlowFile with the content set to the given string (in UTF-8 format), with no attributes,
@@ -410,7 +410,7 @@ public interface TestRunner {
      *
      * @param data to enqueue
      */
-    void enqueue(String data);
+    MockFlowFile enqueue(String data);
 
     /**
      * Copies the content from the given byte array into memory and creates a
@@ -420,7 +420,7 @@ public interface TestRunner {
      * @param data to enqueue
      * @param attributes to use for enqueued item
      */
-    void enqueue(byte[] data, Map<String, String> attributes);
+    MockFlowFile enqueue(byte[] data, Map<String, String> attributes);
 
     /**
      * Creates a FlowFile with the content set to the given string (in UTF-8 format), with the given attributes,
@@ -429,7 +429,7 @@ public interface TestRunner {
      * @param data to enqueue
      * @param attributes to use for enqueued item
      */
-    void enqueue(String data, Map<String, String> attributes);
+    MockFlowFile enqueue(String data, Map<String, String> attributes);
 
     /**
      * Reads the content from the given {@link InputStream} into memory and
@@ -438,7 +438,7 @@ public interface TestRunner {
      *
      * @param data to source data from
      */
-    void enqueue(InputStream data);
+    MockFlowFile enqueue(InputStream data);
 
     /**
      * Reads the content from the given {@link InputStream} into memory and
@@ -448,7 +448,7 @@ public interface TestRunner {
      * @param data source of data
      * @param attributes to use for flow files
      */
-    void enqueue(InputStream data, Map<String, String> attributes);
+    MockFlowFile enqueue(InputStream data, Map<String, String> attributes);
 
     /**
      * Copies the contents of the given {@link MockFlowFile} into a byte array

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
index 847f8a4..e859f94 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_0_10.java
@@ -21,17 +21,14 @@ 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 javax.xml.bind.DatatypeConverter;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -39,13 +36,12 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.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.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -53,17 +49,18 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURITY_PROTOCOL;
+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.10 Consumer API. "
+@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.10.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 mean time"
-        + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.")
-@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.10"})
+        + " 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_10.")
+@Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.10.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."),
+            + "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")
@@ -75,22 +72,16 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURI
         + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
 public class ConsumeKafka_0_10 extends AbstractProcessor {
 
-    private static final long FIVE_MB = 5L * 1024L * 1024L;
-
     static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
 
     static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
 
     static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group");
 
-    static final 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 PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
             .name("topic")
             .displayName("Topic Name(s)")
-            .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma seperated.")
+            .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)
@@ -136,6 +127,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
                     + "will result in a single FlowFile which  "
                     + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS")
             .build();
+
     static final PropertyDescriptor MAX_POLL_RECORDS = new PropertyDescriptor.Builder()
             .name("max.poll.records")
             .displayName("Max Poll Records")
@@ -145,6 +137,20 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
             .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 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.")
@@ -153,8 +159,8 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
     static final List<PropertyDescriptor> DESCRIPTORS;
     static final Set<Relationship> RELATIONSHIPS;
 
-    private volatile byte[] demarcatorBytes = null;
     private volatile ConsumerPool consumerPool = null;
+    private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
 
     static {
         List<PropertyDescriptor> descriptors = new ArrayList<>();
@@ -165,6 +171,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
         descriptors.add(KEY_ATTRIBUTE_ENCODING);
         descriptors.add(MESSAGE_DEMARCATOR);
         descriptors.add(MAX_POLL_RECORDS);
+        descriptors.add(MAX_UNCOMMITTED_TIME);
         DESCRIPTORS = Collections.unmodifiableList(descriptors);
         RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
     }
@@ -179,16 +186,8 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
         return DESCRIPTORS;
     }
 
-    @OnScheduled
-    public void prepareProcessing(final ProcessContext context) {
-        this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet()
-                ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
-                : null;
-    }
-
     @OnStopped
     public void close() {
-        demarcatorBytes = null;
         final ConsumerPool pool = consumerPool;
         consumerPool = null;
         if (pool != null) {
@@ -215,9 +214,21 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
             return pool;
         }
 
-        final Map<String, String> props = new HashMap<>();
+        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_10.MESSAGE_DEMARCATOR).isSet()
+                ? context.getProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
+                : null;
+        final Map<String, Object> props = new HashMap<>();
         KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
-        final String topicListing = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue();
+        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_10.TOPICS).evaluateAttributeExpressions().getValue();
         final List<String> topics = new ArrayList<>();
         for (final String topic : topicListing.split(",", 100)) {
             final String trimmedName = topic.trim();
@@ -225,212 +236,78 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
                 topics.add(trimmedName);
             }
         }
-        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
-        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-        return consumerPool = createConsumerPool(context.getMaxConcurrentTasks(), topics, props, getLogger());
-    }
+        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).getValue();
 
-    protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) {
-        return new ConsumerPool(maxLeases, topics, props, log);
+        return new ConsumerPool(maxLeases, demarcator, props, topics, maxUncommittedTime, keyEncoding, securityProtocol, bootstrapServers, log);
     }
 
-    @Override
-    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-        final long startTimeNanos = System.nanoTime();
-        final ConsumerPool pool = getConsumerPool(context);
-        if (pool == null) {
-            context.yield();
-            return;
-        }
-        final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap = new HashMap<>();
-
-        try (final ConsumerLease lease = pool.obtainConsumer()) {
+    @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 {
-                if (lease == null) {
-                    context.yield();
-                    return;
-                }
-
-                final boolean foundData = gatherDataFromKafka(lease, partitionRecordMap, context);
-                if (!foundData) {
-                    session.rollback();
-                    return;
-                }
-
-                writeSessionData(context, session, partitionRecordMap, startTimeNanos);
-                //At-least once commit handling (if order is reversed it is at-most once)
-                session.commit();
-                commitOffsets(lease, partitionRecordMap);
-            } catch (final KafkaException ke) {
-                lease.poison();
-                getLogger().error("Problem while accessing kafka consumer " + ke, ke);
-                context.yield();
-                session.rollback();
+                Thread.sleep(100L);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                return;
             }
         }
-    }
 
-    private void commitOffsets(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap) {
-        final Map<TopicPartition, OffsetAndMetadata> partOffsetMap = new HashMap<>();
-        partitionRecordMap.entrySet().stream()
-                .filter(entry -> !entry.getValue().isEmpty())
-                .forEach((entry) -> {
-                    long maxOffset = entry.getValue().stream()
-                            .mapToLong(record -> record.offset())
-                            .max()
-                            .getAsLong();
-                    partOffsetMap.put(entry.getKey(), new OffsetAndMetadata(maxOffset + 1L));
-                });
-        lease.commitOffsets(partOffsetMap);
-    }
+        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++;
+            }
 
-    private void writeSessionData(
-            final ProcessContext context, final ProcessSession session,
-            final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap,
-            final long startTimeNanos) {
-        if (demarcatorBytes != null) {
-            partitionRecordMap.entrySet().stream()
-                    .filter(entry -> !entry.getValue().isEmpty())
-                    .forEach(entry -> {
-                        writeData(context, session, entry.getValue(), startTimeNanos);
-                    });
-        } else {
-            partitionRecordMap.entrySet().stream()
-                    .filter(entry -> !entry.getValue().isEmpty())
-                    .flatMap(entry -> entry.getValue().stream())
-                    .forEach(record -> {
-                        writeData(context, session, Collections.singletonList(record), startTimeNanos);
-                    });
+            getLogger().info("Woke up {} consumers", new Object[] {count});
         }
-    }
 
-    private String encodeKafkaKey(final byte[] key, final String encoding) {
-        if (key == null) {
-            return null;
-        }
+        activeLeases.clear();
+    }
 
-        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
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final ConsumerPool pool = getConsumerPool(context);
+        if (pool == null) {
+            context.yield();
+            return;
         }
-    }
 
-    private void writeData(final ProcessContext context, final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final long startTimeNanos) {
-        final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0);
-        final String offset = String.valueOf(firstRecord.offset());
-        final String keyValue = encodeKafkaKey(firstRecord.key(), context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue());
-        final String topic = firstRecord.topic();
-        final String partition = String.valueOf(firstRecord.partition());
-        FlowFile flowFile = session.create();
-        flowFile = session.write(flowFile, out -> {
-            boolean useDemarcator = false;
-            for (final ConsumerRecord<byte[], byte[]> record : records) {
-                if (useDemarcator) {
-                    out.write(demarcatorBytes);
-                }
-                out.write(record.value());
-                useDemarcator = true;
+        try (final ConsumerLease lease = pool.obtainConsumer(session)) {
+            if (lease == null) {
+                context.yield();
+                return;
             }
-        });
-        final Map<String, String> kafkaAttrs = new HashMap<>();
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, offset);
-        if (keyValue != null && records.size() == 1) {
-            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, keyValue);
-        }
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, partition);
-        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, topic);
-        if (records.size() > 1) {
-            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(records.size()));
-        }
-        flowFile = session.putAllAttributes(flowFile, kafkaAttrs);
-        final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos);
-        final String transitUri = KafkaProcessorUtils.buildTransitURI(
-                context.getProperty(SECURITY_PROTOCOL).getValue(),
-                context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(),
-                topic);
-        session.getProvenanceReporter().receive(flowFile, transitUri, executionDurationMillis);
-        this.getLogger().debug("Created {} containing {} messages from Kafka topic {}, partition {}, starting offset {} in {} millis",
-                new Object[]{flowFile, records.size(), topic, partition, offset, executionDurationMillis});
-        session.transfer(flowFile, REL_SUCCESS);
-    }
 
-    /**
-     * Populates the given partitionRecordMap with new records until we poll
-     * that returns no records or until we have enough data. It is important to
-     * ensure we keep items grouped by their topic and partition so that when we
-     * bundle them we bundle them intelligently and so that we can set offsets
-     * properly even across multiple poll calls.
-     */
-    private boolean gatherDataFromKafka(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, ProcessContext context) {
-        final long startNanos = System.nanoTime();
-        boolean foundData = false;
-        ConsumerRecords<byte[], byte[]> records;
-        final int maxRecords = context.getProperty(MAX_POLL_RECORDS).asInteger();
-        do {
-            records = lease.poll();
-
-            for (final TopicPartition partition : records.partitions()) {
-                List<ConsumerRecord<byte[], byte[]>> currList = partitionRecordMap.get(partition);
-                if (currList == null) {
-                    currList = new ArrayList<>();
-                    partitionRecordMap.put(partition, currList);
+            activeLeases.add(lease);
+            try {
+                while (this.isScheduled() && lease.continuePolling()) {
+                    lease.poll();
                 }
-                currList.addAll(records.records(partition));
-                if (currList.size() > 0) {
-                    foundData = true;
+                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);
             }
-            //If we received data and we still want to get more
-        } while (!records.isEmpty() && !checkIfGatheredEnoughData(partitionRecordMap, maxRecords, startNanos));
-        return foundData;
-    }
-
-    /**
-     * Determines if we have enough data as-is and should move on.
-     *
-     * @return true if we've been gathering for more than 500 ms or if we're
-     * demarcating and have more than 50 flowfiles worth or if we're per message
-     * and have more than 2000 flowfiles or if totalMessageSize is greater than
-     * two megabytes; false otherwise
-     *
-     * Implementation note: 500 millis and 5 MB are magic numbers. These may
-     * need to be tuned. They get at how often offsets will get committed to
-     * kafka relative to how many records will get buffered into memory in a
-     * poll call before writing to repos.
-     */
-    private boolean checkIfGatheredEnoughData(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, final int maxRecords, final long startTimeNanos) {
-
-        final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos);
-
-        if (durationMillis > 500) {
-            return true;
-        }
-
-        int topicPartitionsFilled = 0;
-        int totalRecords = 0;
-        long totalRecordSize = 0;
-
-        for (final List<ConsumerRecord<byte[], byte[]>> recordList : partitionRecordMap.values()) {
-            if (!recordList.isEmpty()) {
-                topicPartitionsFilled++;
-            }
-            totalRecords += recordList.size();
-            for (final ConsumerRecord<byte[], byte[]> rec : recordList) {
-                totalRecordSize += rec.value().length;
-            }
-        }
-
-        if (demarcatorBytes != null && demarcatorBytes.length > 0) {
-            return topicPartitionsFilled > 50;
-        } else if (totalRecordSize > FIVE_MB) {
-            return true;
-        } else {
-            return totalRecords > maxRecords;
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
index b954eba..97ebfc6 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
@@ -17,11 +17,28 @@
 package org.apache.nifi.processors.kafka.pubsub;
 
 import java.io.Closeable;
+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.concurrent.TimeUnit;
+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.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10.REL_SUCCESS;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
 
 /**
  * This class represents a lease to access a Kafka Consumer object. The lease is
@@ -30,15 +47,108 @@ import org.apache.kafka.common.TopicPartition;
  * the lease will be returned to the pool for future use by others. A given
  * lease may only belong to a single thread a time.
  */
-public interface ConsumerLease extends Closeable {
+public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListener {
+
+    private final long maxWaitMillis;
+    private final Consumer<byte[], byte[]> kafkaConsumer;
+    private final ComponentLog logger;
+    private final byte[] demarcatorBytes;
+    private final String keyEncoding;
+    private final String securityProtocol;
+    private final String bootstrapServers;
+    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<TopicPartition, BundleTracker> bundleMap = new HashMap<>();
+    private final Map<TopicPartition, OffsetAndMetadata> uncommittedOffsetsMap = new HashMap<>();
+    private long leaseStartNanos = -1;
+    private boolean lastPollEmpty = false;
+    private int totalFlowFiles = 0;
+
+    ConsumerLease(
+            final long maxWaitMillis,
+            final Consumer<byte[], byte[]> kafkaConsumer,
+            final byte[] demarcatorBytes,
+            final String keyEncoding,
+            final String securityProtocol,
+            final String bootstrapServers,
+            final ComponentLog logger) {
+        this.maxWaitMillis = maxWaitMillis;
+        this.kafkaConsumer = kafkaConsumer;
+        this.demarcatorBytes = demarcatorBytes;
+        this.keyEncoding = keyEncoding;
+        this.securityProtocol = securityProtocol;
+        this.bootstrapServers = bootstrapServers;
+        this.logger = logger;
+    }
+
+    /**
+     * 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;
+        totalFlowFiles = 0;
+    }
 
     /**
-     * Executes a poll on the underlying Kafka Consumer.
+     * 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.
      *
-     * @return ConsumerRecords retrieved in the poll.
-     * @throws KafkaException if issue occurs talking to underlying resource.
+     * @param partitions partitions being reassigned
+     */
+    @Override
+    public void onPartitionsRevoked(final Collection<TopicPartition> 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<TopicPartition> 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.
      */
-    ConsumerRecords<byte[], byte[]> poll() throws KafkaException;
+    void poll() {
+        /**
+         * Implementation note: If we take too long (30 secs?) between kafka
+         * poll calls and our own record processing to any subsequent poll calls
+         * or the commit we can run into a situation where the commit will
+         * succeed to the session but fail on committing offsets. This is
+         * apparently different than the Kafka scenario of electing to rebalance
+         * for other reasons but in this case is due a session timeout. It
+         * appears Kafka KIP-62 aims to offer more control over the meaning of
+         * various timeouts. If we do run into this case it could result in
+         * duplicates.
+         */
+        try {
+            final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(10);
+            lastPollEmpty = records.count() == 0;
+            processRecords(records);
+        } catch (final Throwable t) {
+            this.poison();
+            throw t;
+        }
+    }
 
     /**
      * Notifies Kafka to commit the offsets for the specified topic/partition
@@ -47,22 +157,251 @@ public interface ConsumerLease extends Closeable {
      * kafka client to collect more data from Kafka before committing the
      * offsets.
      *
-     * @param offsets offsets
-     * @throws KafkaException if issue occurs talking to underlying resource.
+     * if false then we didn't do anything and should probably yield if true
+     * then we committed new data
+     *
      */
-    void commitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) throws KafkaException;
+    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<FlowFile> bundledFlowFiles = getBundles();
+            if (!bundledFlowFiles.isEmpty()) {
+                getProcessSession().transfer(bundledFlowFiles, REL_SUCCESS);
+            }
+            getProcessSession().commit();
+            kafkaConsumer.commitSync(uncommittedOffsetsMap);
+            resetInternalState();
+            return true;
+        } 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;
+        }
+    }
 
     /**
-     * Notifies that this lease is poisoned and should not be reused.
+     * 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
      */
-    void poison();
+    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 totalFlowFiles < 15000;//admittedlly a magic number - good candidate for processor property
+        }
+    }
 
     /**
-     * Notifies that this lease is to be returned. The pool may optionally reuse
-     * this lease with another client. No further references by the caller
-     * should occur after calling close.
+     * 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
-    void close();
+    public void close() {
+        resetInternalState();
+    }
+
+    public abstract ProcessSession getProcessSession();
+
+    private void processRecords(final ConsumerRecords<byte[], byte[]> records) {
+
+        records.partitions().stream().forEach(partition -> {
+            List<ConsumerRecord<byte[], byte[]>> 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();
+                uncommittedOffsetsMap.put(partition, new OffsetAndMetadata(maxOffset + 1L));
+
+                //write records to content repository and session
+                if (demarcatorBytes == null) {
+                    totalFlowFiles += messages.size();
+                    messages.stream().forEach(message -> {
+                        writeData(getProcessSession(), message, partition);
+                    });
+                } else {
+                    writeData(getProcessSession(), messages, partition);
+                }
+            }
+        });
+    }
+
+    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<FlowFile> getBundles() {
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        for (final BundleTracker tracker : bundleMap.values()) {
+            populateAttributes(tracker);
+            flowFiles.add(tracker.flowFile);
+        }
+        return flowFiles;
+    }
+
+    private void writeData(final ProcessSession session, ConsumerRecord<byte[], byte[]> record, final TopicPartition topicPartition) {
+        FlowFile flowFile = session.create();
+        final BundleTracker tracker = new BundleTracker(record, topicPartition, keyEncoding);
+        tracker.incrementRecordCount(1);
+        flowFile = session.write(flowFile, out -> {
+            out.write(record.value());
+        });
+        tracker.updateFlowFile(flowFile);
+        populateAttributes(tracker);
+        session.transfer(tracker.flowFile, REL_SUCCESS);
+    }
+
+    private void writeData(final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final TopicPartition topicPartition) {
+        final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0);
+        final boolean demarcateFirstRecord;
+        BundleTracker tracker = bundleMap.get(topicPartition);
+        FlowFile flowFile;
+        if (tracker == null) {
+            tracker = new BundleTracker(firstRecord, topicPartition, keyEncoding);
+            flowFile = session.create();
+            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(records.size());
+        flowFile = session.append(flowFile, out -> {
+            boolean useDemarcator = demarcateFirstRecord;
+            for (final ConsumerRecord<byte[], byte[]> record : records) {
+                if (useDemarcator) {
+                    out.write(demarcatorBytes);
+                }
+                out.write(record.value());
+                useDemarcator = true;
+            }
+        });
+        tracker.updateFlowFile(flowFile);
+        bundleMap.put(topicPartition, tracker);
+    }
+
+    private void populateAttributes(final BundleTracker tracker) {
+        final Map<String, String> 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) {
+            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_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;
+        FlowFile flowFile;
+        long totalRecords = 0;
+
+        private BundleTracker(final ConsumerRecord<byte[], byte[]> initialRecord, final TopicPartition topicPartition, final String keyEncoding) {
+            this.initialOffset = initialRecord.offset();
+            this.partition = topicPartition.partition();
+            this.topic = topicPartition.topic();
+            this.key = encodeKafkaKey(initialRecord.key(), keyEncoding);
+        }
+
+        private void incrementRecordCount(final long count) {
+            totalRecords += count;
+        }
+
+        private void updateFlowFile(final FlowFile flowFile) {
+            this.flowFile = flowFile;
+        }
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
index 3f20b8f..baacdc7 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
@@ -21,18 +21,15 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.nifi.logging.ComponentLog;
 
 import java.io.Closeable;
-import java.util.ArrayDeque;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.atomic.AtomicLong;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
+import org.apache.nifi.processor.ProcessSession;
 
 /**
  * A pool of Kafka Consumers for a given topic. Consumers can be obtained by
@@ -41,176 +38,118 @@ import org.apache.kafka.common.TopicPartition;
  */
 public class ConsumerPool implements Closeable {
 
-    private final AtomicInteger activeLeaseCount = new AtomicInteger(0);
-    private final int maxLeases;
-    private final Queue<ConsumerLease> consumerLeases;
+    private final BlockingQueue<SimpleConsumerLease> pooledLeases;
     private final List<String> topics;
     private final Map<String, Object> 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 AtomicLong consumerCreatedCountRef = new AtomicLong();
     private final AtomicLong consumerClosedCountRef = new AtomicLong();
     private final AtomicLong leasesObtainedCountRef = new AtomicLong();
-    private final AtomicLong productivePollCountRef = new AtomicLong();
-    private final AtomicLong unproductivePollCountRef = new AtomicLong();
 
     /**
      * Creates a pool of KafkaConsumer objects that will grow up to the maximum
-     * indicated leases. Consumers are lazily initialized.
+     * 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 maxLeases maximum number of active leases in the pool
-     * @param topics the topics to consume from
-     * @param kafkaProperties the properties for each consumer
+     * @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 maxLeases, final List<String> topics, final Map<String, String> kafkaProperties, final ComponentLog logger) {
-        this.maxLeases = maxLeases;
-        if (maxLeases <= 0) {
-            throw new IllegalArgumentException("Max leases value must be greather than zero.");
-        }
+    public ConsumerPool(
+            final int maxConcurrentLeases,
+            final byte[] demarcator,
+        final Map<String, Object> kafkaProperties,
+            final List<String> topics,
+            final long maxWaitMillis,
+            final String keyEncoding,
+            final String securityProtocol,
+            final String bootstrapServers,
+            final ComponentLog logger) {
+        this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases);
+        this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
-        if (topics == null || topics.isEmpty()) {
-            throw new IllegalArgumentException("Must have a list of one or more topics");
-        }
-        this.topics = topics;
-        this.kafkaProperties = new HashMap<>(kafkaProperties);
-        this.consumerLeases = new ArrayDeque<>();
+        this.demarcatorBytes = demarcator;
+        this.keyEncoding = keyEncoding;
+        this.securityProtocol = securityProtocol;
+        this.bootstrapServers = bootstrapServers;
+        this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties);
+        this.topics = Collections.unmodifiableList(topics);
     }
 
     /**
-     * Obtains a consumer from the pool if one is available
+     * Obtains a consumer from the pool if one is available or lazily
+     * initializes a new one if deemed necessary.
      *
-     * @return consumer from the pool
-     * @throws IllegalArgumentException if pool already contains
+     * @param session the session for which the consumer lease will be
+     * associated
+     * @return consumer to use or null if not available or necessary
      */
-    public ConsumerLease obtainConsumer() {
-        final ConsumerLease lease;
-        final int activeLeases;
-        synchronized (this) {
-            lease = consumerLeases.poll();
-            activeLeases = activeLeaseCount.get();
-        }
-        if (lease == null && activeLeases >= maxLeases) {
-            logger.warn("No available consumers and cannot create any as max consumer leases limit reached - verify pool settings");
-            return null;
+    public ConsumerLease obtainConsumer(final ProcessSession session) {
+        SimpleConsumerLease lease = pooledLeases.poll();
+        if (lease == null) {
+            final Consumer<byte[], byte[]> 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.
+             */
+            consumer.subscribe(topics, lease);
         }
+        lease.setProcessSession(session);
         leasesObtainedCountRef.incrementAndGet();
-        return (lease == null) ? createConsumer() : lease;
+        return lease;
     }
 
+    /**
+     * Exposed as protected method for easier unit testing
+     *
+     * @return consumer
+     * @throws KafkaException if unable to subscribe to the given topics
+     */
     protected Consumer<byte[], byte[]> createKafkaConsumer() {
         return new KafkaConsumer<>(kafkaProperties);
     }
 
-    private ConsumerLease createConsumer() {
-        final Consumer<byte[], byte[]> kafkaConsumer = createKafkaConsumer();
-        consumerCreatedCountRef.incrementAndGet();
-        try {
-            kafkaConsumer.subscribe(topics);
-        } catch (final KafkaException kex) {
-            try {
-                kafkaConsumer.close();
-                consumerClosedCountRef.incrementAndGet();
-            } catch (final Exception ex) {
-                consumerClosedCountRef.incrementAndGet();
-                //ignore
-            }
-            throw kex;
-        }
-
-        final ConsumerLease lease = new ConsumerLease() {
-
-            private volatile boolean poisoned = false;
-            private volatile boolean closed = false;
-
-            @Override
-            public ConsumerRecords<byte[], byte[]> poll() {
-
-                if (poisoned) {
-                    throw new KafkaException("The consumer is poisoned and should no longer be used");
-                }
-
-                try {
-                    final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(50);
-                    if (records.isEmpty()) {
-                        unproductivePollCountRef.incrementAndGet();
-                    } else {
-                        productivePollCountRef.incrementAndGet();
-                    }
-                    return records;
-                } catch (final KafkaException kex) {
-                    logger.warn("Unable to poll from Kafka consumer so will poison and close this " + kafkaConsumer, kex);
-                    poison();
-                    close();
-                    throw kex;
-                }
-            }
-
-            @Override
-            public void commitOffsets(final Map<TopicPartition, OffsetAndMetadata> offsets) {
-
-                if (poisoned) {
-                    throw new KafkaException("The consumer is poisoned and should no longer be used");
-                }
-                try {
-                    kafkaConsumer.commitSync(offsets);
-                } catch (final KafkaException kex) {
-                    logger.warn("Unable to commit kafka consumer offsets so will poison and close this " + kafkaConsumer, kex);
-                    poison();
-                    close();
-                    throw kex;
-                }
-            }
-
-            @Override
-            public void close() {
-                if (closed) {
-                    return;
-                }
-                if (poisoned || activeLeaseCount.get() > maxLeases) {
-                    closeConsumer(kafkaConsumer);
-                    activeLeaseCount.decrementAndGet();
-                    closed = true;
-                } else {
-                    final boolean added;
-                    synchronized (ConsumerPool.this) {
-                        added = consumerLeases.offer(this);
-                    }
-                    if (!added) {
-                        closeConsumer(kafkaConsumer);
-                        activeLeaseCount.decrementAndGet();
-                    }
-                }
-            }
-
-            @Override
-            public void poison() {
-                poisoned = true;
-            }
-        };
-        activeLeaseCount.incrementAndGet();
-        return lease;
-    }
-
     /**
-     * Closes all consumers in the pool. Can be safely recalled.
+     * Closes all consumers in the pool. Can be safely called repeatedly.
      */
     @Override
     public void close() {
-        final List<ConsumerLease> leases = new ArrayList<>();
-        synchronized (this) {
-            ConsumerLease lease = null;
-            while ((lease = consumerLeases.poll()) != null) {
-                leases.add(lease);
-            }
-        }
-        for (final ConsumerLease lease : leases) {
-            lease.poison();
-            lease.close();
-        }
+        final List<SimpleConsumerLease> leases = new ArrayList<>();
+        pooledLeases.drainTo(leases);
+        leases.stream().forEach((lease) -> {
+            lease.close(true);
+        });
     }
 
-    private void closeConsumer(final Consumer consumer) {
+    private void closeConsumer(final Consumer<?, ?> consumer) {
+        consumerClosedCountRef.incrementAndGet();
         try {
             consumer.unsubscribe();
         } catch (Exception e) {
@@ -219,15 +158,55 @@ public class ConsumerPool implements Closeable {
 
         try {
             consumer.close();
-            consumerClosedCountRef.incrementAndGet();
         } catch (Exception e) {
-            consumerClosedCountRef.incrementAndGet();
             logger.warn("Failed while closing " + consumer, e);
         }
     }
 
     PoolStats getPoolStats() {
-        return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get(), productivePollCountRef.get(), unproductivePollCountRef.get());
+        return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get());
+    }
+
+    private class SimpleConsumerLease extends ConsumerLease {
+
+        private final Consumer<byte[], byte[]> consumer;
+        private volatile ProcessSession session;
+        private volatile boolean closedConsumer;
+
+        private SimpleConsumerLease(final Consumer<byte[], byte[]> consumer) {
+            super(maxWaitMillis, consumer, demarcatorBytes, keyEncoding, securityProtocol, bootstrapServers, logger);
+            this.consumer = consumer;
+        }
+
+        void setProcessSession(final ProcessSession session) {
+            this.session = session;
+        }
+
+        @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);
+            }
+            if (forceClose || isPoisoned() || !pooledLeases.offer(this)) {
+                closedConsumer = true;
+                closeConsumer(consumer);
+            }
+        }
     }
 
     static final class PoolStats {
@@ -235,30 +214,22 @@ public class ConsumerPool implements Closeable {
         final long consumerCreatedCount;
         final long consumerClosedCount;
         final long leasesObtainedCount;
-        final long productivePollCount;
-        final long unproductivePollCount;
 
         PoolStats(
                 final long consumerCreatedCount,
                 final long consumerClosedCount,
-                final long leasesObtainedCount,
-                final long productivePollCount,
-                final long unproductivePollCount
+                final long leasesObtainedCount
         ) {
             this.consumerCreatedCount = consumerCreatedCount;
             this.consumerClosedCount = consumerClosedCount;
             this.leasesObtainedCount = leasesObtainedCount;
-            this.productivePollCount = productivePollCount;
-            this.unproductivePollCount = unproductivePollCount;
         }
 
         @Override
         public String toString() {
             return "Created Consumers [" + consumerCreatedCount + "]\n"
                     + "Closed Consumers  [" + consumerClosedCount + "]\n"
-                    + "Leases Obtained   [" + leasesObtainedCount + "]\n"
-                    + "Productive Polls  [" + productivePollCount + "]\n"
-                    + "Unproductive Polls  [" + unproductivePollCount + "]\n";
+                    + "Leases Obtained   [" + leasesObtainedCount + "]\n";
         }
 
     }


[15/20] nifi git commit: NIFI-2777: NIFI-2856: - Only performing response merging when the node is the cluster cooridinator even if there is a single response. - Fixing PropertyDescriptor merging to ensure the 'choosen' descriptor is included in map of a

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/0b1d15a7/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
deleted file mode 100644
index 03aa08a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMergerSpec.groovy
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster.coordination.http
-
-import com.sun.jersey.api.client.ClientResponse
-import org.apache.nifi.cluster.manager.NodeResponse
-import org.apache.nifi.cluster.protocol.NodeIdentifier
-import org.apache.nifi.util.NiFiProperties
-import org.apache.nifi.web.api.dto.AccessPolicyDTO
-import org.apache.nifi.web.api.dto.ConnectionDTO
-import org.apache.nifi.web.api.dto.ControllerConfigurationDTO
-import org.apache.nifi.web.api.dto.FunnelDTO
-import org.apache.nifi.web.api.dto.LabelDTO
-import org.apache.nifi.web.api.dto.PermissionsDTO
-import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO
-import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO
-import org.apache.nifi.web.api.entity.ConnectionEntity
-import org.apache.nifi.web.api.entity.ConnectionsEntity
-import org.apache.nifi.web.api.entity.ControllerConfigurationEntity
-import org.apache.nifi.web.api.entity.FunnelEntity
-import org.apache.nifi.web.api.entity.FunnelsEntity
-import org.apache.nifi.web.api.entity.LabelEntity
-import org.apache.nifi.web.api.entity.LabelsEntity
-import org.codehaus.jackson.map.ObjectMapper
-import org.codehaus.jackson.map.SerializationConfig
-import org.codehaus.jackson.map.annotate.JsonSerialize
-import org.codehaus.jackson.xc.JaxbAnnotationIntrospector
-import spock.lang.Specification
-import spock.lang.Unroll
-
-@Unroll
-class StandardHttpResponseMergerSpec extends Specification {
-
-    def setup() {
-        def propFile = StandardHttpResponseMergerSpec.class.getResource("/conf/nifi.properties").getFile()
-        System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
-    }
-
-    def cleanup() {
-        System.clearProperty NiFiProperties.PROPERTIES_FILE_PATH
-    }
-
-    def "MergeResponses: mixed HTTP GET response statuses, expecting #expectedStatus"() {
-        given:
-        def responseMerger = new StandardHttpResponseMerger(NiFiProperties.createBasicNiFiProperties(null,null))
-        def requestUri = new URI('http://server/resource')
-        def requestId = UUID.randomUUID().toString()
-        def Map<ClientResponse, Map<String, Integer>> mockToRequestEntity = [:]
-        def nodeResponseSet = nodeResponseData.collect {
-            int n = it.node
-            def clientResponse = Mock(ClientResponse)
-            mockToRequestEntity.put clientResponse, it
-            new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, clientResponse, 500L, requestId)
-        } as Set
-
-        when:
-        def returnedResponse = responseMerger.mergeResponses(requestUri, 'get', nodeResponseSet).getStatus()
-
-        then:
-        mockToRequestEntity.entrySet().forEach {
-            ClientResponse mockClientResponse = it.key
-            _ * mockClientResponse.getStatus() >> it.value.status
-        }
-        0 * _
-        returnedResponse == expectedStatus
-
-        where:
-        nodeResponseData                                                                || expectedStatus
-        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 401]] as Set || 401
-        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 403]] as Set || 403
-        [[node: 1, status: 200], [node: 2, status: 403], [node: 3, status: 500]] as Set || 403
-        [[node: 1, status: 200], [node: 2, status: 200], [node: 3, status: 500]] as Set || 500
-    }
-
-    def "MergeResponses: #responseEntities.size() HTTP 200 #httpMethod responses for #requestUriPart"() {
-        given: "json serialization setup"
-        def mapper = new ObjectMapper();
-        def jaxbIntrospector = new JaxbAnnotationIntrospector();
-        def SerializationConfig serializationConfig = mapper.getSerializationConfig();
-        mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(JsonSerialize.Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector));
-
-        and: "setup of the data to be used in the test"
-        def responseMerger = new StandardHttpResponseMerger(NiFiProperties.createBasicNiFiProperties(null,null))
-        def requestUri = new URI("http://server/$requestUriPart")
-        def requestId = UUID.randomUUID().toString()
-        def Map<ClientResponse, Object> mockToRequestEntity = [:]
-        def n = 0
-        def nodeResponseSet = responseEntities.collect {
-            ++n
-            def clientResponse = Mock(ClientResponse)
-            mockToRequestEntity.put clientResponse, it
-            new NodeResponse(new NodeIdentifier("cluster-node-$n", 'addr', n, 'sktaddr', n * 10, 'stsaddr', n * 100, n * 1000, false, null), "get", requestUri, clientResponse, 500L, requestId)
-        } as Set
-
-        when:
-        def returnedResponse = responseMerger.mergeResponses(requestUri, httpMethod, nodeResponseSet)
-
-        then:
-        mockToRequestEntity.entrySet().forEach {
-            ClientResponse mockClientResponse = it.key
-            def entity = it.value
-            _ * mockClientResponse.getStatus() >> 200
-            1 * mockClientResponse.getEntity(_) >> entity
-        }
-        responseEntities.size() == mockToRequestEntity.size()
-        0 * _
-        def returnedJson = mapper.writeValueAsString(returnedResponse.getUpdatedEntity())
-        def expectedJson = mapper.writeValueAsString(expectedEntity)
-        returnedJson == expectedJson
-
-        where:
-        requestUriPart                                             | httpMethod | responseEntities                                                                                     ||
-                expectedEntity
-        'nifi-api/controller/config'                               | 'get'      | [
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))]                                                      ||
-                // expectedEntity
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))
-        'nifi-api/controller/config'                               | 'put'      | [
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10)),
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: true),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))]                                                      ||
-                // expectedEntity
-                new ControllerConfigurationEntity(permissions: new PermissionsDTO(canRead: true, canWrite: false),
-                        component: new ControllerConfigurationDTO(maxEventDrivenThreadCount: 10, maxTimerDrivenThreadCount: 10))
-        "nifi-api/process-groups/${UUID.randomUUID()}/connections" | 'get'      | [
-                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status: new
-                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())] as Set),
-                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status: new
-                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 100)))] as Set),
-                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status: new
-                        ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 500)), component: new ConnectionDTO())] as Set)] ||
-                // expectedEntity
-                new ConnectionsEntity(connections: [new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
-                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 900,
-                                input: '0 (900 bytes)', output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))] as Set)
-        "nifi-api/process-groups/${UUID.randomUUID()}/connections" | 'post'     | [
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO()),
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300))),
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())]      ||
-                // expectedEntity
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
-                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 900, input: '0 (900 bytes)',
-                                output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))
-        "nifi-api/connections/${UUID.randomUUID()}"                | 'get'      | [
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 400)), component: new ConnectionDTO()),
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300))),
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), status:
-                        new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 300)), component: new ConnectionDTO())]      ||
-                // expectedEntity
-                new ConnectionEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false),
-                        status: new ConnectionStatusDTO(aggregateSnapshot: new ConnectionStatusSnapshotDTO(bytesIn: 1000,
-                                input: '0 (1,000 bytes)', output: '0 (0 bytes)', queued: '0 (0 bytes)', queuedSize: '0 bytes', queuedCount: 0)))
-        "nifi-api/process-groups/${UUID.randomUUID()}/labels" | 'get'      | [
-                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())] as Set),
-                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set),
-                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())] as Set)] ||
-                // expectedEntity
-                new LabelsEntity(labels: [new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set)
-        "nifi-api/process-groups/${UUID.randomUUID()}/labels" | 'post'     | [
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO()),
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())]      ||
-                // expectedEntity
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
-        "nifi-api/labels/${UUID.randomUUID()}"                | 'get'      | [
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO()),
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new LabelDTO())]      ||
-                // expectedEntity
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
-        "nifi-api/process-groups/${UUID.randomUUID()}/funnels" | 'get'      | [
-                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())] as Set),
-                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set),
-                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())] as Set)] ||
-                // expectedEntity
-                new FunnelsEntity(funnels: [new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))] as Set)
-        "nifi-api/process-groups/${UUID.randomUUID()}/funnels" | 'post'     | [
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO()),
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())]      ||
-                // expectedEntity
-                new LabelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
-        "nifi-api/funnels/${UUID.randomUUID()}"                | 'get'      | [
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO()),
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false)),
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: true, canWrite: true), component: new FunnelDTO())]      ||
-                // expectedEntity
-                new FunnelEntity(id: '1', permissions: new PermissionsDTO(canRead: false, canWrite: false))
-    }
-}