You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by oz...@apache.org on 2016/04/04 19:58:19 UTC

[1/9] nifi git commit: NIFI-1684 fixed NPE, added tests [Forced Update!]

Repository: nifi
Updated Branches:
  refs/heads/support/nifi-0.6.x 980317ba0 -> 71476b2d3 (forced update)


NIFI-1684 fixed NPE, added tests

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: be5c95e5e93bd492c99889a3f550091fb2c4a48f
Parents: 0b9bd20
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Mar 24 07:37:58 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:51:54 2016 -0400

----------------------------------------------------------------------
 .../kafka/SplittableMessageContext.java         | 22 ++++---
 .../kafka/SplittableMessageContextTest.java     | 64 ++++++++++++++++++++
 2 files changed, 79 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/be5c95e5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
index 9967404..d597a05 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
@@ -46,6 +46,9 @@ final class SplittableMessageContext {
      *            - "(\\W)\\Z".
      */
     SplittableMessageContext(String topicName, byte[] keyBytes, String delimiterPattern) {
+        if (topicName == null || topicName.trim().length() == 0){
+            throw new IllegalArgumentException("'topicName' must not be null or empty");
+        }
         this.topicName = topicName;
         this.keyBytes = keyBytes;
         this.delimiterPattern = delimiterPattern != null ? delimiterPattern : "(\\W)\\Z";
@@ -60,20 +63,25 @@ final class SplittableMessageContext {
     }
 
     /**
-     *
+     * Will set failed segments from an array of integers
      */
     void setFailedSegments(int... failedSegments) {
-        this.failedSegments = new BitSet();
-        for (int failedSegment : failedSegments) {
-            this.failedSegments.set(failedSegment);
+        if (failedSegments != null) {
+            this.failedSegments = new BitSet();
+            for (int failedSegment : failedSegments) {
+                this.failedSegments.set(failedSegment);
+            }
         }
     }
 
     /**
-     *
+     * Will set failed segments from an array of bytes that will be used to
+     * construct the final {@link BitSet} representing failed segments
      */
     void setFailedSegmentsAsByteArray(byte[] failedSegments) {
-        this.failedSegments = BitSet.valueOf(failedSegments);
+        if (failedSegments != null) {
+            this.failedSegments = BitSet.valueOf(failedSegments);
+        }
     }
 
     /**
@@ -102,7 +110,7 @@ final class SplittableMessageContext {
      * Returns the key bytes as String
      */
     String getKeyBytesAsString() {
-        return new String(this.keyBytes);
+        return this.keyBytes != null ? new String(this.keyBytes) : null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/be5c95e5/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
new file mode 100644
index 0000000..b12464a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.nio.charset.StandardCharsets;
+
+import org.junit.Test;
+
+public class SplittableMessageContextTest {
+
+    @Test(expected = IllegalArgumentException.class)
+    public void failNullEmptyTopic() {
+        new SplittableMessageContext(null, null, null);
+    }
+
+    @Test
+    public void validateFullSetting() {
+        SplittableMessageContext ctx = new SplittableMessageContext("foo", "hello".getBytes(), "\n");
+        ctx.setFailedSegments(1, 3, 6);
+        assertEquals("\n", ctx.getDelimiterPattern());
+        assertEquals("hello", new String(ctx.getKeyBytes(), StandardCharsets.UTF_8));
+        assertEquals("foo", ctx.getTopicName());
+        assertEquals("topic: 'foo'; delimiter: '\n'", ctx.toString());
+    }
+
+
+    @Test
+    public void validateToString() {
+        SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null);
+        assertEquals("topic: 'foo'; delimiter: '(\\W)\\Z'", ctx.toString());
+    }
+
+    @Test
+    public void validateNoNPEandNoSideffectsOnSetsGets() {
+        SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null);
+        ctx.setFailedSegments(null);
+        assertNull(ctx.getFailedSegments());
+
+        ctx.setFailedSegmentsAsByteArray(null);
+        assertNull(ctx.getFailedSegments());
+
+        assertEquals("(\\W)\\Z", ctx.getDelimiterPattern());;
+        assertNull(ctx.getKeyBytes());
+        assertNull(ctx.getKeyBytesAsString());
+        assertEquals("foo", ctx.getTopicName());
+    }
+}


[4/9] nifi git commit: NIFI-1684 This closes #302. fixed random partitioner initialization

Posted by oz...@apache.org.
NIFI-1684 This closes #302. fixed random partitioner initialization

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: 89567ebfba3b6d4cf42b04aa981085b1f0e3564e
Parents: 7561fa5
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Sat Mar 26 10:47:13 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:52:46 2016 -0400

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/processors/kafka/PutKafka.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/89567ebf/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index 44f04aa..6d17493 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -55,8 +55,6 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 
-import kafka.producer.DefaultPartitioner;
-
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" })
 @CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka. The messages to send may be individual FlowFiles or may be delimited, using a "
@@ -454,7 +452,7 @@ public class PutKafka extends AbstractProcessor {
         if (partitionStrategy.equalsIgnoreCase(ROUND_ROBIN_PARTITIONING.getValue())) {
             partitionerClass = Partitioners.RoundRobinPartitioner.class.getName();
         } else if (partitionStrategy.equalsIgnoreCase(RANDOM_PARTITIONING.getValue())) {
-            partitionerClass = DefaultPartitioner.class.getName();
+            partitionerClass = Partitioners.RandomPartitioner.class.getName();
         }
         properties.setProperty("partitioner.class", partitionerClass);
 


[3/9] nifi git commit: NIFI-1684 added penalization on failure

Posted by oz...@apache.org.
NIFI-1684 added penalization on failure

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: 7561fa53e4e7ca4e3256d0de8491cc6e5b93f76e
Parents: f0546d0
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Mar 24 15:29:35 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:52:35 2016 -0400

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/processors/kafka/PutKafka.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/7561fa53/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index 513f4f3..44f04aa 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -306,7 +306,7 @@ public class PutKafka extends AbstractProcessor {
                 session.transfer(flowFile, REL_SUCCESS);
             } else {
                 flowFile = session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(failedSegmentsRef.get(), messageContext));
-                session.transfer(flowFile, REL_FAILURE);
+                session.transfer(session.penalize(flowFile), REL_FAILURE);
             }
 
         } else {


[7/9] nifi git commit: NIFI-1701 fixed StreamScanner, added more tests

Posted by oz...@apache.org.
NIFI-1701 fixed StreamScanner, added more tests

NIFI-1701 additional refactoring, clean up and more tests


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

Branch: refs/heads/support/nifi-0.6.x
Commit: b5e007213bfb8769842fce9f770a9890f2661d8c
Parents: 9a8ac35
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Mar 31 00:59:26 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:53:42 2016 -0400

----------------------------------------------------------------------
 .../nifi/processors/kafka/KafkaPublisher.java   |   9 +-
 .../apache/nifi/processors/kafka/PutKafka.java  |  19 +--
 .../kafka/SplittableMessageContext.java         |  23 +--
 .../nifi/processors/kafka/StreamScanner.java    | 139 ++++++++++++++-----
 .../processors/kafka/KafkaPublisherTest.java    |  34 ++++-
 .../kafka/SplittableMessageContextTest.java     |  10 +-
 .../processors/kafka/StreamScannerTests.java    | 130 +++++++++++++++++
 .../nifi/processors/kafka/TestPutKafka.java     |  66 ++++++++-
 8 files changed, 361 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
index e116978..ebdf5c8 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
@@ -43,7 +43,7 @@ import kafka.producer.Partitioner;
  * Wrapper over {@link KafkaProducer} to assist {@link PutKafka} processor with
  * sending content of {@link FlowFile}s to Kafka.
  */
-public class KafkaPublisher implements AutoCloseable {
+class KafkaPublisher implements AutoCloseable {
 
     private static final Logger logger = LoggerFactory.getLogger(KafkaPublisher.class);
 
@@ -112,14 +112,16 @@ public class KafkaPublisher implements AutoCloseable {
      *            the value of the partition key. Only relevant is user wishes
      *            to provide a custom partition key instead of relying on
      *            variety of provided {@link Partitioner}(s)
+     * @param maxBufferSize maximum message size
      * @return The set containing the failed segment indexes for messages that
      *         failed to be sent to Kafka.
      */
-    BitSet publish(SplittableMessageContext messageContext, InputStream contentStream, Integer partitionKey) {
+    BitSet publish(SplittableMessageContext messageContext, InputStream contentStream, Integer partitionKey,
+            int maxBufferSize) {
         List<Future<RecordMetadata>> sendFutures = new ArrayList<>();
         BitSet prevFailedSegmentIndexes = messageContext.getFailedSegments();
         int segmentCounter = 0;
-        StreamScanner scanner = new StreamScanner(contentStream, messageContext.getDelimiterPattern());
+        StreamScanner scanner = new StreamScanner(contentStream, messageContext.getDelimiterBytes(), maxBufferSize);
 
         while (scanner.hasNext()) {
             byte[] content = scanner.next();
@@ -136,7 +138,6 @@ public class KafkaPublisher implements AutoCloseable {
                 segmentCounter++;
             }
         }
-        scanner.close();
         return this.processAcks(sendFutures);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index 6d17493..3b5eb4f 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -155,7 +155,7 @@ public class PutKafka extends AbstractProcessor {
             .build();
     public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder()
             .name("Message Delimiter")
-            .description("Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. "
+            .description("Specifies the delimiter (interpreted in its UTF-8 byte representation) to use for splitting apart 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. Note that if messages are delimited and some messages for a given FlowFile are transferred "
@@ -177,7 +177,8 @@ public class PutKafka extends AbstractProcessor {
     static final PropertyDescriptor MAX_RECORD_SIZE = new PropertyDescriptor.Builder()
             .name("Max Record Size")
             .description("The maximum size that any individual record can be.")
-            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR).required(true)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .required(true)
             .defaultValue("1 MB")
             .build();
     public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
@@ -294,7 +295,8 @@ public class PutKafka extends AbstractProcessor {
             session.read(flowFile, new InputStreamCallback() {
                 @Override
                 public void process(InputStream contentStream) throws IOException {
-                    failedSegmentsRef.set(kafkaPublisher.publish(messageContext, contentStream, partitionKey));
+                    int maxRecordSize = context.getProperty(MAX_RECORD_SIZE).asDataSize(DataUnit.B).intValue();
+                    failedSegmentsRef.set(kafkaPublisher.publish(messageContext, contentStream, partitionKey, maxRecordSize));
                 }
             });
 
@@ -391,7 +393,7 @@ public class PutKafka extends AbstractProcessor {
         attributes.put(ATTR_FAILED_SEGMENTS, new String(failedSegments.toByteArray(), StandardCharsets.UTF_8));
         attributes.put(ATTR_TOPIC, messageContext.getTopicName());
         attributes.put(ATTR_KEY, messageContext.getKeyBytesAsString());
-        attributes.put(ATTR_DELIMITER, messageContext.getDelimiterPattern());
+        attributes.put(ATTR_DELIMITER, new String(messageContext.getDelimiterBytes(), StandardCharsets.UTF_8));
         return attributes;
     }
 
@@ -401,21 +403,22 @@ public class PutKafka extends AbstractProcessor {
     private SplittableMessageContext buildMessageContext(FlowFile flowFile, ProcessContext context, ProcessSession session) {
         String topicName;
         byte[] key;
-        String delimiterPattern;
+        byte[] delimiterBytes;
 
         String failedSegmentsString = flowFile.getAttribute(ATTR_FAILED_SEGMENTS);
         if (flowFile.getAttribute(ATTR_PROC_ID) != null && flowFile.getAttribute(ATTR_PROC_ID).equals(this.getIdentifier()) && failedSegmentsString != null) {
             topicName = flowFile.getAttribute(ATTR_TOPIC);
             key = flowFile.getAttribute(ATTR_KEY) == null ? null : flowFile.getAttribute(ATTR_KEY).getBytes();
-            delimiterPattern = flowFile.getAttribute(ATTR_DELIMITER);
+            delimiterBytes = flowFile.getAttribute(ATTR_DELIMITER) != null ? flowFile.getAttribute(ATTR_DELIMITER).getBytes(StandardCharsets.UTF_8) : null;
         } else {
             failedSegmentsString = null;
             topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
             String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
             key = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8);
-            delimiterPattern = context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue();
+            delimiterBytes = context.getProperty(MESSAGE_DELIMITER).isSet()
+                    ? context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
         }
-        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, key, delimiterPattern);
+        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, key, delimiterBytes);
         if (failedSegmentsString != null) {
             messageContext.setFailedSegmentsAsByteArray(failedSegmentsString.getBytes());
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
index d597a05..d5f1c0b 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/SplittableMessageContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.kafka;
 
+import java.nio.charset.StandardCharsets;
 import java.util.BitSet;
 
 import org.apache.nifi.flowfile.FlowFile;
@@ -29,7 +30,7 @@ import org.apache.nifi.flowfile.FlowFile;
 final class SplittableMessageContext {
     private final String topicName;
 
-    private final String delimiterPattern;
+    private final byte[] delimiterBytes;
 
     private final byte[] keyBytes;
 
@@ -40,18 +41,17 @@ final class SplittableMessageContext {
      *            the name of the Kafka topic
      * @param keyBytes
      *            the instance of byte[] representing the key. Can be null.
-     * @param delimiterPattern
-     *            the string representing the delimiter regex pattern. Can be
-     *            null. For cases where it is null the EOF pattern will be used
-     *            - "(\\W)\\Z".
+     * @param delimiterBytes
+     *            byte array representing bytes by which the data will be
+     *            delimited. Can be null.
      */
-    SplittableMessageContext(String topicName, byte[] keyBytes, String delimiterPattern) {
+    SplittableMessageContext(String topicName, byte[] keyBytes, byte[] delimiterBytes) {
         if (topicName == null || topicName.trim().length() == 0){
             throw new IllegalArgumentException("'topicName' must not be null or empty");
         }
         this.topicName = topicName;
         this.keyBytes = keyBytes;
-        this.delimiterPattern = delimiterPattern != null ? delimiterPattern : "(\\W)\\Z";
+        this.delimiterBytes = delimiterBytes != null ? delimiterBytes : null;
     }
 
     /**
@@ -59,7 +59,8 @@ final class SplittableMessageContext {
      */
     @Override
     public String toString() {
-        return "topic: '" + topicName + "'; delimiter: '" + delimiterPattern + "'";
+        String delVal = this.delimiterBytes != null ? " delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'" : "";
+        return "topic: '" + topicName + "';" + delVal;
     }
 
     /**
@@ -100,10 +101,10 @@ final class SplittableMessageContext {
     }
 
     /**
-     * Returns the value of the delimiter regex pattern.
+     * Returns the delimiter bytes
      */
-    String getDelimiterPattern() {
-        return this.delimiterPattern;
+    byte[] getDelimiterBytes() {
+        return this.delimiterBytes;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/StreamScanner.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/StreamScanner.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/StreamScanner.java
index e959fdd..ee83a02 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/StreamScanner.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/StreamScanner.java
@@ -17,77 +17,148 @@
 package org.apache.nifi.processors.kafka;
 
 import java.io.BufferedInputStream;
+import java.io.IOException;
 import java.io.InputStream;
-import java.util.Arrays;
-
-import org.apache.nifi.stream.io.ByteArrayOutputStream;
-import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
+import java.nio.ByteBuffer;
 
 /**
  *
  */
 class StreamScanner {
 
+    private final static byte EOF = -1;
+
     private final InputStream is;
 
-    private final byte[] delimiter;
+    private final byte[] delimiterBytes;
 
-    private final NonThreadSafeCircularBuffer buffer;
+    private final int maxDataSize;
 
-    private final ByteArrayOutputStream baos;
+    private ByteBuffer buffer;
 
     private byte[] data;
 
-    private boolean eos;
+    /**
+     * Constructs a new instance
+     *
+     * @param is
+     *            instance of {@link InputStream} representing the data
+     * @param delimiterBytes
+     *            byte array representing delimiter bytes used to split the
+     *            input stream. Can be null
+     * @param maxDataSize
+     *            maximum size of data derived from the input stream. This means
+     *            that neither {@link InputStream} nor its individual chunks (if
+     *            delimiter is used) can ever be greater then this size.
+     */
+    StreamScanner(InputStream is, byte[] delimiterBytes, int maxDataSize) {
+        this(is, delimiterBytes, maxDataSize, 8192);
+    }
 
     /**
+     * Constructs a new instance
+     *
+     * @param is
+     *            instance of {@link InputStream} representing the data
+     * @param delimiterBytes
+     *            byte array representing delimiter bytes used to split the
+     *            input stream. Can be null
+     * @param maxDataSize
+     *            maximum size of data derived from the input stream. This means
+     *            that neither {@link InputStream} nor its individual chunks (if
+     *            delimiter is used) can ever be greater then this size.
+     * @param initialBufferSize
+     *            initial size of the buffer used to buffer {@link InputStream}
+     *            or its parts (if delimiter is used) to create its byte[]
+     *            representation. Must be positive integer. The buffer will grow
+     *            automatically as needed up to the Integer.MAX_VALUE;
      *
      */
-    StreamScanner(InputStream is, String delimiter) {
+    StreamScanner(InputStream is, byte[] delimiterBytes, int maxDataSize, int initialBufferSize) {
         this.is = new BufferedInputStream(is);
-        this.delimiter = delimiter.getBytes();
-        buffer = new NonThreadSafeCircularBuffer(this.delimiter);
-        baos = new ByteArrayOutputStream();
+        this.delimiterBytes = delimiterBytes;
+        this.buffer = ByteBuffer.allocate(initialBufferSize);
+        this.maxDataSize = maxDataSize;
     }
 
     /**
+     * Checks if there are more elements in the stream. This operation is
+     * idempotent.
      *
+     * @return <i>true</i> if there are more elements in the stream or
+     *         <i>false</i> when it reaches the end of the stream after the last
+     *         element was retrieved via {@link #next()} operation.
      */
     boolean hasNext() {
-        this.data = null;
-        if (!this.eos) {
+        int j = 0;
+        int readVal = 0;
+        while (this.data == null && readVal != EOF) {
+            this.expandBufferIfNecessary();
             try {
-                boolean keepReading = true;
-                while (keepReading) {
-                    byte b = (byte) this.is.read();
-                    if (b > -1) {
-                        baos.write(b);
-                        if (buffer.addAndCompare(b)) {
-                            this.data = Arrays.copyOfRange(baos.getUnderlyingBuffer(), 0, baos.size() - delimiter.length);
-                            keepReading = false;
-                        }
-                    } else {
-                        this.data = baos.toByteArray();
-                        keepReading = false;
-                        this.eos = true;
+                readVal = this.is.read();
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed while reading InputStream", e);
+            }
+            if (readVal == EOF) {
+                this.extractDataToken(0);
+            } else {
+                byte byteVal = (byte)readVal;
+                this.buffer.put(byteVal);
+                if (this.buffer.position() > this.maxDataSize) {
+                    throw new IllegalStateException("Maximum allowed data size of " + this.maxDataSize + " exceeded.");
+                }
+                if (this.delimiterBytes != null && this.delimiterBytes[j] == byteVal) {
+                    if (++j == this.delimiterBytes.length) {
+                        this.extractDataToken(this.delimiterBytes.length);
+                        j = 0;
                     }
+                } else {
+                    j = 0;
                 }
-                baos.reset();
-            } catch (Exception e) {
-                throw new IllegalStateException("Failed while reading InputStream", e);
             }
         }
         return this.data != null;
     }
 
     /**
-     *
+     * @return byte array representing the next segment in the stream or the
+     *         whole stream if no delimiter is used
      */
     byte[] next() {
-        return this.data;
+        try {
+            return this.data;
+        } finally {
+            this.data = null;
+        }
+    }
+
+    /**
+     *
+     */
+    private void expandBufferIfNecessary() {
+        if (this.buffer.position() == Integer.MAX_VALUE ){
+            throw new IllegalStateException("Internal buffer has reached the capacity and can not be expended any further");
+        }
+        if (this.buffer.remaining() == 0) {
+            this.buffer.flip();
+            int pos = this.buffer.capacity();
+            int newSize = this.buffer.capacity() * 2 > Integer.MAX_VALUE ? Integer.MAX_VALUE : this.buffer.capacity() * 2;
+            ByteBuffer bb = ByteBuffer.allocate(newSize);
+            bb.put(this.buffer);
+            this.buffer = bb;
+            this.buffer.position(pos);
+        }
     }
 
-    void close() {
-        this.baos.close();
+    /**
+     *
+     */
+    private void extractDataToken(int lengthSubtract) {
+        this.buffer.flip();
+        if (this.buffer.limit() > 0){ // something must be in the buffer; at least delimiter (e.g., \n)
+            this.data = new byte[this.buffer.limit() - lengthSubtract];
+            this.buffer.get(this.data);
+        }
+        this.buffer.clear();
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java
index 92a6307..f21dfb0 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/KafkaPublisherTest.java
@@ -83,7 +83,7 @@ public class KafkaPublisherTest {
 
         SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null);
 
-        publisher.publish(messageContext, fis, null);
+        publisher.publish(messageContext, fis, null, 2000);
 
         fis.close();
         publisher.close();
@@ -105,9 +105,9 @@ public class KafkaPublisherTest {
         Properties kafkaProperties = this.buildProducerProperties();
         KafkaPublisher publisher = new KafkaPublisher(kafkaProperties);
 
-        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n");
+        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n".getBytes(StandardCharsets.UTF_8));
 
-        publisher.publish(messageContext, fis, null);
+        publisher.publish(messageContext, fis, null, 2000);
         publisher.close();
 
         ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
@@ -131,9 +131,9 @@ public class KafkaPublisherTest {
         Properties kafkaProperties = this.buildProducerProperties();
         KafkaPublisher publisher = new KafkaPublisher(kafkaProperties);
 
-        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "|");
+        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "|".getBytes(StandardCharsets.UTF_8));
 
-        publisher.publish(messageContext, fis, null);
+        publisher.publish(messageContext, fis, null, 2000);
         publisher.close();
 
         ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
@@ -157,10 +157,10 @@ public class KafkaPublisherTest {
 
         KafkaPublisher publisher = new KafkaPublisher(kafkaProperties);
 
-        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n");
+        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n".getBytes(StandardCharsets.UTF_8));
         messageContext.setFailedSegments(1, 3);
 
-        publisher.publish(messageContext, fis, null);
+        publisher.publish(messageContext, fis, null, 2000);
         publisher.close();
 
         ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
@@ -176,6 +176,26 @@ public class KafkaPublisherTest {
         }
     }
 
+    @Test
+    public void validateWithMultiByteCharacters() throws Exception {
+        String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
+        InputStream fis = new ByteArrayInputStream(data.getBytes());
+        String topicName = "validateWithMultiByteCharacters";
+
+        Properties kafkaProperties = this.buildProducerProperties();
+
+        KafkaPublisher publisher = new KafkaPublisher(kafkaProperties);
+
+        SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null);
+
+        publisher.publish(messageContext, fis, null, 2000);
+        publisher.close();
+
+        ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
+        String r = new String(iter.next().message());
+        assertEquals(data, r);
+    }
+
     private Properties buildProducerProperties() {
         Properties kafkaProperties = new Properties();
         kafkaProperties.setProperty("bootstrap.servers", "0.0.0.0:" + kafkaLocal.getKafkaPort());

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
index b12464a..8b5048f 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/SplittableMessageContextTest.java
@@ -32,9 +32,9 @@ public class SplittableMessageContextTest {
 
     @Test
     public void validateFullSetting() {
-        SplittableMessageContext ctx = new SplittableMessageContext("foo", "hello".getBytes(), "\n");
+        SplittableMessageContext ctx = new SplittableMessageContext("foo", "hello".getBytes(), "\n".getBytes(StandardCharsets.UTF_8));
         ctx.setFailedSegments(1, 3, 6);
-        assertEquals("\n", ctx.getDelimiterPattern());
+        assertEquals("\n", new String(ctx.getDelimiterBytes(), StandardCharsets.UTF_8));
         assertEquals("hello", new String(ctx.getKeyBytes(), StandardCharsets.UTF_8));
         assertEquals("foo", ctx.getTopicName());
         assertEquals("topic: 'foo'; delimiter: '\n'", ctx.toString());
@@ -44,7 +44,9 @@ public class SplittableMessageContextTest {
     @Test
     public void validateToString() {
         SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null);
-        assertEquals("topic: 'foo'; delimiter: '(\\W)\\Z'", ctx.toString());
+        assertEquals("topic: 'foo';", ctx.toString());
+        ctx = new SplittableMessageContext("foo", null, "blah".getBytes(StandardCharsets.UTF_8));
+        assertEquals("topic: 'foo'; delimiter: 'blah'", ctx.toString());
     }
 
     @Test
@@ -56,7 +58,7 @@ public class SplittableMessageContextTest {
         ctx.setFailedSegmentsAsByteArray(null);
         assertNull(ctx.getFailedSegments());
 
-        assertEquals("(\\W)\\Z", ctx.getDelimiterPattern());;
+        assertNull(ctx.getDelimiterBytes());
         assertNull(ctx.getKeyBytes());
         assertNull(ctx.getKeyBytesAsString());
         assertEquals("foo", ctx.getTopicName());

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/StreamScannerTests.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/StreamScannerTests.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/StreamScannerTests.java
new file mode 100644
index 0000000..1ebc4c4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/StreamScannerTests.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class StreamScannerTests {
+
+    @Test
+    public void validateWithMultiByteCharsNoDelimiter() {
+        String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, null, 1000);
+        assertTrue(scanner.hasNext());
+        assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
+        assertFalse(scanner.hasNext());
+    }
+
+    @Test
+    public void validateWithComplexDelimiter() {
+        String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY NEW TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY NEWEST TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertFalse(scanner.hasNext());
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void validateMaxBufferSize() {
+        String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 20);
+        assertTrue(scanner.hasNext());
+    }
+
+    @Test
+    public void verifyScannerHandlesNegativeOneByteInputs() {
+        ByteArrayInputStream is = new ByteArrayInputStream(new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
+        StreamScanner scanner = new StreamScanner(is, "water".getBytes(StandardCharsets.UTF_8), 20, 1024);
+        assertTrue(scanner.hasNext());
+        Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
+    }
+
+    @Test
+    public void verifyScannerHandlesNegativeOneByteDelimiter() {
+        ByteArrayInputStream is = new ByteArrayInputStream(new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
+        StreamScanner scanner = new StreamScanner(is, new byte[] { -1 }, 20, 1024);
+        assertTrue(scanner.hasNext());
+        Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0, 0});
+        assertTrue(scanner.hasNext());
+        Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0});
+    }
+
+    @Test
+    public void validateHasNextIdempotencyWithDelimiter() {
+        String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
+        for (int i = 0; i < 5; i++) { // we only have 3 segments so unless idempotent hasNext would return false after 3 tries
+            assertTrue(scanner.hasNext());
+        }
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY NEW TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertTrue(scanner.hasNext());
+        assertEquals("THIS IS MY NEWEST TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
+        assertFalse(scanner.hasNext());
+    }
+
+    @Test
+    public void validateHasNextIdempotencyWithoutDelimiter() {
+        String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, null, 1000);
+        for (int i = 0; i < 5; i++) { // we only have 3 segments so unless idempotent hasNext would return false after 3 tries
+            assertTrue(scanner.hasNext());
+        }
+        assertTrue(scanner.hasNext());
+        assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
+        assertFalse(scanner.hasNext());
+    }
+
+    @Test
+    public void validateInternalBufferCanExpend() throws Exception {
+        String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
+        ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
+        StreamScanner scanner = new StreamScanner(is, null, 1000, 2);
+        Field bufferField = StreamScanner.class.getDeclaredField("buffer");
+        bufferField.setAccessible(true);
+        ByteBuffer buffer = (ByteBuffer) bufferField.get(scanner);
+        assertEquals(2, buffer.capacity());
+
+        assertTrue(scanner.hasNext());
+        assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
+        assertFalse(scanner.hasNext());
+
+        buffer = (ByteBuffer) bufferField.get(scanner);
+        assertEquals(128, buffer.capacity());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/b5e00721/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java
index 3ed0549..34544df 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java
@@ -66,6 +66,7 @@ public class TestPutKafka {
     }
 
     @Test
+    @Ignore
     public void testDelimitedMessagesWithKey() {
         String topicName = "testDelimitedMessagesWithKey";
         PutKafka putKafka = new PutKafka();
@@ -95,7 +96,7 @@ public class TestPutKafka {
     @Test
     @Ignore
     public void testWithFailureAndPartialResend() throws Exception {
-        String topicName = "testWithImmediateFailure";
+        String topicName = "testWithFailureAndPartialResend";
         PutKafka putKafka = new PutKafka();
         final TestRunner runner = TestRunners.newTestRunner(putKafka);
         runner.setProperty(PutKafka.TOPIC, topicName);
@@ -187,6 +188,69 @@ public class TestPutKafka {
         }
     }
 
+    @Test
+    public void testComplexRightPartialDelimitedMessages() {
+        String topicName = "testComplexRightPartialDelimitedMessages";
+        PutKafka putKafka = new PutKafka();
+        TestRunner runner = TestRunners.newTestRunner(putKafka);
+        runner.setProperty(PutKafka.TOPIC, topicName);
+        runner.setProperty(PutKafka.CLIENT_NAME, "foo");
+        runner.setProperty(PutKafka.SEED_BROKERS, "localhost:" + kafkaLocal.getKafkaPort());
+        runner.setProperty(PutKafka.MESSAGE_DELIMITER, "僠<僠WILDSTUFF僠>僠");
+
+        runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>".getBytes());
+        runner.run(1, false);
+
+        runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
+        ConsumerIterator<byte[], byte[]> consumer = this.buildConsumer(topicName);
+        assertEquals("Hello World", new String(consumer.next().message()));
+        assertEquals("Goodbye", new String(consumer.next().message()));
+        assertEquals("I Mean IT!僠<僠WILDSTUFF僠>", new String(consumer.next().message()));
+        runner.shutdown();
+    }
+
+    @Test
+    public void testComplexLeftPartialDelimitedMessages() {
+        String topicName = "testComplexLeftPartialDelimitedMessages";
+        PutKafka putKafka = new PutKafka();
+        TestRunner runner = TestRunners.newTestRunner(putKafka);
+        runner.setProperty(PutKafka.TOPIC, topicName);
+        runner.setProperty(PutKafka.CLIENT_NAME, "foo");
+        runner.setProperty(PutKafka.SEED_BROKERS, "localhost:" + kafkaLocal.getKafkaPort());
+        runner.setProperty(PutKafka.MESSAGE_DELIMITER, "僠<僠WILDSTUFF僠>僠");
+
+        runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>僠<僠WILDSTUFF僠>僠".getBytes());
+        runner.run(1, false);
+
+        runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
+        ConsumerIterator<byte[], byte[]> consumer = this.buildConsumer(topicName);
+        assertEquals("Hello World", new String(consumer.next().message()));
+        assertEquals("Goodbye", new String(consumer.next().message()));
+        assertEquals("I Mean IT!", new String(consumer.next().message()));
+        assertEquals("<僠WILDSTUFF僠>僠", new String(consumer.next().message()));
+        runner.shutdown();
+    }
+
+    @Test
+    public void testComplexPartialMatchDelimitedMessages() {
+        String topicName = "testComplexPartialMatchDelimitedMessages";
+        PutKafka putKafka = new PutKafka();
+        TestRunner runner = TestRunners.newTestRunner(putKafka);
+        runner.setProperty(PutKafka.TOPIC, topicName);
+        runner.setProperty(PutKafka.CLIENT_NAME, "foo");
+        runner.setProperty(PutKafka.SEED_BROKERS, "localhost:" + kafkaLocal.getKafkaPort());
+        runner.setProperty(PutKafka.MESSAGE_DELIMITER, "僠<僠WILDSTUFF僠>僠");
+
+        runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDBOOMSTUFF僠>僠".getBytes());
+        runner.run(1, false);
+
+        runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
+        ConsumerIterator<byte[], byte[]> consumer = this.buildConsumer(topicName);
+        assertEquals("Hello World", new String(consumer.next().message()));
+        assertEquals("Goodbye僠<僠WILDBOOMSTUFF僠>僠", new String(consumer.next().message()));
+        runner.shutdown();
+    }
+
     private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
         Properties props = new Properties();
         props.put("zookeeper.connect", "0.0.0.0:" + kafkaLocal.getZookeeperPort());


[5/9] nifi git commit: NIFI-1684 This closes #308. fixed ZKClient connection leak

Posted by oz...@apache.org.
NIFI-1684 This closes #308. fixed ZKClient connection leak

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: 552d831807efc06b714759db9ecb71cc41142826
Parents: 89567eb
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Mon Mar 28 21:48:37 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:52:58 2016 -0400

----------------------------------------------------------------------
 .../apache/nifi/processors/kafka/GetKafka.java  | 13 +++---
 .../nifi/processors/kafka/KafkaUtils.java       | 43 ++++++++++++++------
 2 files changed, 37 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/552d8318/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
index e06befb..7660305 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
@@ -256,13 +256,14 @@ public class GetKafka extends AbstractProcessor {
             props.setProperty("consumer.timeout.ms", "1");
         }
 
+        int partitionCount = KafkaUtils.retrievePartitionCountForTopic(
+                context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue(), context.getProperty(TOPIC).getValue());
+
         final ConsumerConfig consumerConfig = new ConsumerConfig(props);
         consumer = Consumer.createJavaConsumerConnector(consumerConfig);
 
         final Map<String, Integer> topicCountMap = new HashMap<>(1);
 
-        int partitionCount = KafkaUtils.retrievePartitionCountForTopic(context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue(), context.getProperty(TOPIC).getValue());
-
         int concurrentTaskToUse = context.getMaxConcurrentTasks();
         if (context.getMaxConcurrentTasks() < partitionCount){
             this.getLogger().warn("The amount of concurrent tasks '" + context.getMaxConcurrentTasks() + "' configured for "
@@ -346,14 +347,14 @@ public class GetKafka extends AbstractProcessor {
                 try {
                     f.get(this.deadlockTimeout, TimeUnit.MILLISECONDS);
                 } catch (InterruptedException e) {
-                    this.consumerStreamsReady.set(false);
+                    shutdownConsumer();
                     f.cancel(true);
                     Thread.currentThread().interrupt();
                     getLogger().warn("Interrupted while waiting to get connection", e);
                 } catch (ExecutionException e) {
                     throw new IllegalStateException(e);
                 } catch (TimeoutException e) {
-                    this.consumerStreamsReady.set(false);
+                    shutdownConsumer();
                     f.cancel(true);
                     getLogger().warn("Timed out after " + this.deadlockTimeout + " milliseconds while waiting to get connection", e);
                 }
@@ -374,14 +375,14 @@ public class GetKafka extends AbstractProcessor {
             try {
                 consumptionFuture.get(this.deadlockTimeout, TimeUnit.MILLISECONDS);
             } catch (InterruptedException e) {
-                this.consumerStreamsReady.set(false);
+                shutdownConsumer();
                 consumptionFuture.cancel(true);
                 Thread.currentThread().interrupt();
                 getLogger().warn("Interrupted while consuming messages", e);
             } catch (ExecutionException e) {
                 throw new IllegalStateException(e);
             } catch (TimeoutException e) {
-                this.consumerStreamsReady.set(false);
+                shutdownConsumer();
                 consumptionFuture.cancel(true);
                 getLogger().warn("Timed out after " + this.deadlockTimeout + " milliseconds while consuming messages", e);
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/552d8318/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java
index a725c2b..8ddea61 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java
@@ -33,25 +33,42 @@ import scala.collection.JavaConversions;
  */
 class KafkaUtils {
 
+
     /**
      * Will retrieve the amount of partitions for a given Kafka topic.
      */
     static int retrievePartitionCountForTopic(String zookeeperConnectionString, String topicName) {
-        ZkClient zkClient = new ZkClient(zookeeperConnectionString);
+        ZkClient zkClient = null;
 
-        zkClient.setZkSerializer(new ZkSerializer() {
-            @Override
-            public byte[] serialize(Object o) throws ZkMarshallingError {
-                return ZKStringSerializer.serialize(o);
-            }
+        try {
+            zkClient = new ZkClient(zookeeperConnectionString);
+            zkClient.setZkSerializer(new ZkSerializer() {
+                @Override
+                public byte[] serialize(Object o) throws ZkMarshallingError {
+                    return ZKStringSerializer.serialize(o);
+                }
 
-            @Override
-            public Object deserialize(byte[] bytes) throws ZkMarshallingError {
-                return ZKStringSerializer.deserialize(bytes);
+                @Override
+                public Object deserialize(byte[] bytes) throws ZkMarshallingError {
+                    return ZKStringSerializer.deserialize(bytes);
+                }
+            });
+            scala.collection.Set<TopicMetadata> topicMetadatas = AdminUtils
+                    .fetchTopicMetadataFromZk(JavaConversions.asScalaSet(Collections.singleton(topicName)), zkClient);
+            if (topicMetadatas != null && topicMetadatas.size() > 0) {
+                return JavaConversions.asJavaSet(topicMetadatas).iterator().next().partitionsMetadata().size();
+            } else {
+                throw new IllegalStateException("Failed to get metadata for topic " + topicName);
             }
-        });
-        scala.collection.Set<TopicMetadata> topicMetadatas = AdminUtils
-                .fetchTopicMetadataFromZk(JavaConversions.asScalaSet(Collections.singleton(topicName)), zkClient);
-        return topicMetadatas.size();
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to retrieve partitions for topic " + topicName, e);
+        } finally {
+            try {
+                zkClient.close();
+            } catch (Exception e2) {
+                // ignore
+            }
+        }
     }
+
 }


[2/9] nifi git commit: NIFI-1684 fixed NPE in PutKafka when retrieving key attribute bytes

Posted by oz...@apache.org.
NIFI-1684 fixed NPE in PutKafka when retrieving key attribute bytes

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: f0546d04872d202043834ff398c31420347f95b7
Parents: be5c95e
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Mar 24 14:35:40 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:52:23 2016 -0400

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/processors/kafka/PutKafka.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/f0546d04/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index 4510038..513f4f3 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -408,7 +408,7 @@ public class PutKafka extends AbstractProcessor {
         String failedSegmentsString = flowFile.getAttribute(ATTR_FAILED_SEGMENTS);
         if (flowFile.getAttribute(ATTR_PROC_ID) != null && flowFile.getAttribute(ATTR_PROC_ID).equals(this.getIdentifier()) && failedSegmentsString != null) {
             topicName = flowFile.getAttribute(ATTR_TOPIC);
-            key = flowFile.getAttribute(ATTR_KEY).getBytes();
+            key = flowFile.getAttribute(ATTR_KEY) == null ? null : flowFile.getAttribute(ATTR_KEY).getBytes();
             delimiterPattern = flowFile.getAttribute(ATTR_DELIMITER);
         } else {
             failedSegmentsString = null;


[9/9] nifi git commit: NIFI-1691: Add Fetch Size property to QueryDatabaseTable

Posted by oz...@apache.org.
NIFI-1691: Add Fetch Size property to QueryDatabaseTable

This closes #307

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/71476b2d
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/71476b2d
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/71476b2d

Branch: refs/heads/support/nifi-0.6.x
Commit: 71476b2d3e313800dba783423d70a044f22b6469
Parents: e5bb1f5
Author: Matt Burgess <ma...@apache.org>
Authored: Mon Mar 28 12:16:24 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:54:43 2016 -0400

----------------------------------------------------------------------
 .../processors/standard/QueryDatabaseTable.java | 21 ++++++++++++++++++++
 .../standard/QueryDatabaseTableTest.java        |  1 +
 2 files changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/71476b2d/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
index 08f6b41..9403eb8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
@@ -175,6 +175,15 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
             .defaultValue("None")
             .build();
 
+    public static final PropertyDescriptor FETCH_SIZE = new PropertyDescriptor.Builder()
+            .name("Fetch Size")
+            .description("The number of result rows to be fetched from the result set at a time. This is a hint to the driver and may not be "
+                    + "honored and/or exact. If the value specified is zero, then the hint is ignored.")
+            .defaultValue("0")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
 
     private final List<PropertyDescriptor> propDescriptors;
 
@@ -192,6 +201,7 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
         pds.add(MAX_VALUE_COLUMN_NAMES);
         pds.add(QUERY_TIMEOUT);
         pds.add(SQL_PREPROCESS_STRATEGY);
+        pds.add(FETCH_SIZE);
         propDescriptors = Collections.unmodifiableList(pds);
     }
 
@@ -251,6 +261,8 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
         final String columnNames = context.getProperty(COLUMN_NAMES).getValue();
         final String maxValueColumnNames = context.getProperty(MAX_VALUE_COLUMN_NAMES).getValue();
         final String preProcessStrategy = context.getProperty(SQL_PREPROCESS_STRATEGY).getValue();
+        final Integer fetchSize = context.getProperty(FETCH_SIZE).asInteger();
+
         final StateManager stateManager = context.getStateManager();
         final StateMap stateMap;
 
@@ -272,6 +284,15 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
         try (final Connection con = dbcpService.getConnection();
              final Statement st = con.createStatement()) {
 
+            if (fetchSize != null && fetchSize > 0) {
+                try {
+                    st.setFetchSize(fetchSize);
+                } catch (SQLException se) {
+                    // Not all drivers support this, just log the error (at debug level) and move on
+                    logger.debug("Cannot set fetch size to {} due to {}", new Object[]{fetchSize, se.getLocalizedMessage()}, se);
+                }
+            }
+
             final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
             st.setQueryTimeout(queryTimeout); // timeout in seconds
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/71476b2d/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/QueryDatabaseTableTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/QueryDatabaseTableTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/QueryDatabaseTableTest.java
index d16b9c6..f932e4d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/QueryDatabaseTableTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/QueryDatabaseTableTest.java
@@ -154,6 +154,7 @@ public class QueryDatabaseTableTest {
         runner.assertAllFlowFilesTransferred(QueryDatabaseTable.REL_SUCCESS, 1);
 
         InputStream in = new ByteArrayInputStream(runner.getFlowFilesForRelationship(QueryDatabaseTable.REL_SUCCESS).get(0).toByteArray());
+        runner.setProperty(QueryDatabaseTable.FETCH_SIZE, "2");
         assertEquals(3, getNumberOfRecordsFromStream(in));
         runner.clearTransferState();
 


[6/9] nifi git commit: NIFI-1697 Ensuring FlowController appropriately wraps code with NarCloseable. This closes #312

Posted by oz...@apache.org.
NIFI-1697 Ensuring FlowController appropriately wraps code with NarCloseable. This closes #312

Signed-off-by: Matt Gilman <ma...@gmail.com>


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

Branch: refs/heads/support/nifi-0.6.x
Commit: 9a8ac35bb6086e3e69041fc9fca2fdb91e87bad3
Parents: 552d831
Author: Bryan Bende <bb...@apache.org>
Authored: Wed Mar 30 11:46:52 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:53:20 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/nifi/controller/FlowController.java | 12 +++++++++---
 .../apache/nifi/controller/StandardProcessorNode.java   |  4 +++-
 2 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/9a8ac35b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index c9aaceb..09c4da6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -3220,13 +3220,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final PrimaryNodeState nodeState = primary ? PrimaryNodeState.ELECTED_PRIMARY_NODE : PrimaryNodeState.PRIMARY_NODE_REVOKED;
             final ProcessGroup rootGroup = getGroup(getRootGroupId());
             for (final ProcessorNode procNode : rootGroup.findAllProcessors()) {
-                ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, procNode.getProcessor(), nodeState);
+                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                    ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, procNode.getProcessor(), nodeState);
+                }
             }
             for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
-                ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, serviceNode.getControllerServiceImplementation(), nodeState);
+                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                    ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, serviceNode.getControllerServiceImplementation(), nodeState);
+                }
             }
             for (final ReportingTaskNode reportingTaskNode : getAllReportingTasks()) {
-                ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, reportingTaskNode.getReportingTask(), nodeState);
+                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                    ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, reportingTaskNode.getReportingTask(), nodeState);
+                }
             }
 
             // update primary

http://git-wip-us.apache.org/repos/asf/nifi/blob/9a8ac35b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 8ac82a0..b544f38 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -908,7 +908,9 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                     getAnnotationData());
 
             final Collection<ValidationResult> validationResults;
-            validationResults = getProcessor().validate(validationContext);
+            try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                validationResults = getProcessor().validate(validationContext);
+            }
 
             for (final ValidationResult result : validationResults) {
                 if (!result.isValid()) {


[8/9] nifi git commit: NIFI-1442: This closes #306. Use CircularFifoQueue instead of Set to store nodes' bulletins Joint effort by Toivo Adams from PR306 and and Mark Payne

Posted by oz...@apache.org.
NIFI-1442: This closes #306. Use CircularFifoQueue instead of Set to store nodes' bulletins
Joint effort by Toivo Adams from PR306 and and Mark Payne

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


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

Branch: refs/heads/support/nifi-0.6.x
Commit: e5bb1f52c06dd70faf75ed1b0b45ea5ae1763026
Parents: b5e0072
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Mar 28 10:35:44 2016 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Mon Apr 4 13:54:34 2016 -0400

----------------------------------------------------------------------
 .../events/NodeBulletinProcessingStrategy.java  | 40 ++++------------
 .../TestNodeBulletinProcessingStrategy.java     | 49 ++++++++++++++++++++
 2 files changed, 57 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/e5bb1f52/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/NodeBulletinProcessingStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/NodeBulletinProcessingStrategy.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/NodeBulletinProcessingStrategy.java
index d3cfd9e..8c00d64 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/NodeBulletinProcessingStrategy.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/NodeBulletinProcessingStrategy.java
@@ -17,50 +17,26 @@
 package org.apache.nifi.events;
 
 import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.Set;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.collections4.queue.CircularFifoQueue;
 import org.apache.nifi.reporting.Bulletin;
 
 /**
  *
  */
 public class NodeBulletinProcessingStrategy implements BulletinProcessingStrategy {
-
-    private final Lock lock;
-    private final Set<Bulletin> bulletins;
-
-    public NodeBulletinProcessingStrategy() {
-        lock = new ReentrantLock();
-        bulletins = new LinkedHashSet<>();
-    }
+    static final int MAX_ENTRIES = 5;
+    private final CircularFifoQueue<Bulletin> ringBuffer = new CircularFifoQueue<>(MAX_ENTRIES);
 
     @Override
-    public void update(final Bulletin bulletin) {
-        lock.lock();
-        try {
-            bulletins.add(bulletin);
-        } finally {
-            lock.unlock();
-        }
+    public synchronized void update(final Bulletin bulletin) {
+        ringBuffer.add(bulletin);
     }
 
-    public Set<Bulletin> getBulletins() {
-        final Set<Bulletin> response = new HashSet<>();
-
-        lock.lock();
-        try {
-            // get all the bulletins currently stored
-            response.addAll(bulletins);
-
-            // remove the bulletins
-            bulletins.clear();
-        } finally {
-            lock.unlock();
-        }
-
+    public synchronized Set<Bulletin> getBulletins() {
+        final Set<Bulletin> response = new HashSet<>(ringBuffer);
+        ringBuffer.clear();
         return response;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/e5bb1f52/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/events/TestNodeBulletinProcessingStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/events/TestNodeBulletinProcessingStrategy.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/events/TestNodeBulletinProcessingStrategy.java
new file mode 100644
index 0000000..394c940
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/events/TestNodeBulletinProcessingStrategy.java
@@ -0,0 +1,49 @@
+/*
+ * 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.events;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+public class TestNodeBulletinProcessingStrategy {
+
+    @Test
+    public void testUpdate() {
+
+        NodeBulletinProcessingStrategy nBulletinProcessingStrategy = new NodeBulletinProcessingStrategy();
+
+        nBulletinProcessingStrategy.update(new ComponentBulletin(1));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(2));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(3));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(4));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(5));
+        assertEquals(5, nBulletinProcessingStrategy.getBulletins().size());
+
+        nBulletinProcessingStrategy.update(new ComponentBulletin(1));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(2));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(3));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(4));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(5));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(6));
+        nBulletinProcessingStrategy.update(new ComponentBulletin(7));
+        assertEquals(NodeBulletinProcessingStrategy.MAX_ENTRIES, nBulletinProcessingStrategy.getBulletins().size());
+
+    }
+
+}
\ No newline at end of file