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