NIFI-1701 fixed StreamScanner, added more tests

NIFI-1701 additional refactoring, clean up and more tests
This commit is contained in:
Oleg Zhurakousky 2016-03-31 00:59:26 -04:00 committed by Mark Payne
parent 65b5c1a5f1
commit 25290cedc4
8 changed files with 364 additions and 72 deletions

View File

@ -43,7 +43,7 @@ import kafka.producer.Partitioner;
* Wrapper over {@link KafkaProducer} to assist {@link PutKafka} processor with * Wrapper over {@link KafkaProducer} to assist {@link PutKafka} processor with
* sending content of {@link FlowFile}s to Kafka. * 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); 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 * the value of the partition key. Only relevant is user wishes
* to provide a custom partition key instead of relying on * to provide a custom partition key instead of relying on
* variety of provided {@link Partitioner}(s) * variety of provided {@link Partitioner}(s)
* @param maxBufferSize maximum message size
* @return The set containing the failed segment indexes for messages that * @return The set containing the failed segment indexes for messages that
* failed to be sent to Kafka. * 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<>(); List<Future<RecordMetadata>> sendFutures = new ArrayList<>();
BitSet prevFailedSegmentIndexes = messageContext.getFailedSegments(); BitSet prevFailedSegmentIndexes = messageContext.getFailedSegments();
int segmentCounter = 0; int segmentCounter = 0;
StreamScanner scanner = new StreamScanner(contentStream, messageContext.getDelimiterPattern()); StreamScanner scanner = new StreamScanner(contentStream, messageContext.getDelimiterBytes(), maxBufferSize);
while (scanner.hasNext()) { while (scanner.hasNext()) {
byte[] content = scanner.next(); byte[] content = scanner.next();
@ -136,7 +138,6 @@ public class KafkaPublisher implements AutoCloseable {
segmentCounter++; segmentCounter++;
} }
} }
scanner.close();
return this.processAcks(sendFutures); return this.processAcks(sendFutures);
} }

View File

@ -155,7 +155,7 @@ public class PutKafka extends AbstractProcessor {
.build(); .build();
public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder() public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder()
.name("Message Delimiter") .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, " + "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 " + "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 " + "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() static final PropertyDescriptor MAX_RECORD_SIZE = new PropertyDescriptor.Builder()
.name("Max Record Size") .name("Max Record Size")
.description("The maximum size that any individual record can be.") .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") .defaultValue("1 MB")
.build(); .build();
public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder() public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
@ -294,7 +295,8 @@ public class PutKafka extends AbstractProcessor {
session.read(flowFile, new InputStreamCallback() { session.read(flowFile, new InputStreamCallback() {
@Override @Override
public void process(InputStream contentStream) throws IOException { 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_FAILED_SEGMENTS, new String(failedSegments.toByteArray(), StandardCharsets.UTF_8));
attributes.put(ATTR_TOPIC, messageContext.getTopicName()); attributes.put(ATTR_TOPIC, messageContext.getTopicName());
attributes.put(ATTR_KEY, messageContext.getKeyBytesAsString()); 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; return attributes;
} }
@ -401,21 +403,22 @@ public class PutKafka extends AbstractProcessor {
private SplittableMessageContext buildMessageContext(FlowFile flowFile, ProcessContext context, ProcessSession session) { private SplittableMessageContext buildMessageContext(FlowFile flowFile, ProcessContext context, ProcessSession session) {
String topicName; String topicName;
byte[] key; byte[] key;
String delimiterPattern; byte[] delimiterBytes;
String failedSegmentsString = flowFile.getAttribute(ATTR_FAILED_SEGMENTS); String failedSegmentsString = flowFile.getAttribute(ATTR_FAILED_SEGMENTS);
if (flowFile.getAttribute(ATTR_PROC_ID) != null && flowFile.getAttribute(ATTR_PROC_ID).equals(this.getIdentifier()) && failedSegmentsString != null) { if (flowFile.getAttribute(ATTR_PROC_ID) != null && flowFile.getAttribute(ATTR_PROC_ID).equals(this.getIdentifier()) && failedSegmentsString != null) {
topicName = flowFile.getAttribute(ATTR_TOPIC); topicName = flowFile.getAttribute(ATTR_TOPIC);
key = flowFile.getAttribute(ATTR_KEY) == null ? null : flowFile.getAttribute(ATTR_KEY).getBytes(); 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 { } else {
failedSegmentsString = null; failedSegmentsString = null;
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
key = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8); 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) { if (failedSegmentsString != null) {
messageContext.setFailedSegmentsAsByteArray(failedSegmentsString.getBytes()); messageContext.setFailedSegmentsAsByteArray(failedSegmentsString.getBytes());
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.nifi.processors.kafka; package org.apache.nifi.processors.kafka;
import java.nio.charset.StandardCharsets;
import java.util.BitSet; import java.util.BitSet;
import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFile;
@ -29,7 +30,7 @@ import org.apache.nifi.flowfile.FlowFile;
final class SplittableMessageContext { final class SplittableMessageContext {
private final String topicName; private final String topicName;
private final String delimiterPattern; private final byte[] delimiterBytes;
private final byte[] keyBytes; private final byte[] keyBytes;
@ -40,18 +41,17 @@ final class SplittableMessageContext {
* the name of the Kafka topic * the name of the Kafka topic
* @param keyBytes * @param keyBytes
* the instance of byte[] representing the key. Can be null. * the instance of byte[] representing the key. Can be null.
* @param delimiterPattern * @param delimiterBytes
* the string representing the delimiter regex pattern. Can be * byte array representing bytes by which the data will be
* null. For cases where it is null the EOF pattern will be used * delimited. Can be null.
* - "(\\W)\\Z".
*/ */
SplittableMessageContext(String topicName, byte[] keyBytes, String delimiterPattern) { SplittableMessageContext(String topicName, byte[] keyBytes, byte[] delimiterBytes) {
if (topicName == null || topicName.trim().length() == 0){ if (topicName == null || topicName.trim().length() == 0){
throw new IllegalArgumentException("'topicName' must not be null or empty"); throw new IllegalArgumentException("'topicName' must not be null or empty");
} }
this.topicName = topicName; this.topicName = topicName;
this.keyBytes = keyBytes; this.keyBytes = keyBytes;
this.delimiterPattern = delimiterPattern != null ? delimiterPattern : "(\\W)\\Z"; this.delimiterBytes = delimiterBytes != null ? delimiterBytes : null;
} }
/** /**
@ -59,7 +59,8 @@ final class SplittableMessageContext {
*/ */
@Override @Override
public String toString() { 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() { byte[] getDelimiterBytes() {
return this.delimiterPattern; return this.delimiterBytes;
} }
/** /**

View File

@ -17,77 +17,148 @@
package org.apache.nifi.processors.kafka; package org.apache.nifi.processors.kafka;
import java.io.BufferedInputStream; import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.util.Arrays; import java.nio.ByteBuffer;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
/** /**
* *
*/ */
class StreamScanner { class StreamScanner {
private final static byte EOF = -1;
private final InputStream is; 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 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, String delimiter) { StreamScanner(InputStream is, byte[] delimiterBytes, int maxDataSize) {
this.is = new BufferedInputStream(is); this(is, delimiterBytes, maxDataSize, 8192);
this.delimiter = delimiter.getBytes();
buffer = new NonThreadSafeCircularBuffer(this.delimiter);
baos = new ByteArrayOutputStream();
} }
/** /**
* 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, byte[] delimiterBytes, int maxDataSize, int initialBufferSize) {
this.is = new BufferedInputStream(is);
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() { boolean hasNext() {
this.data = null; int j = 0;
if (!this.eos) { int readVal = 0;
while (this.data == null && readVal != EOF) {
this.expandBufferIfNecessary();
try { try {
boolean keepReading = true; readVal = this.is.read();
while (keepReading) { } catch (IOException e) {
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;
}
}
baos.reset();
} catch (Exception e) {
throw new IllegalStateException("Failed while reading InputStream", 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;
}
}
} }
return this.data != null; 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() { byte[] next() {
return this.data; try {
return this.data;
} finally {
this.data = null;
}
} }
void close() { /**
this.baos.close(); *
*/
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);
}
}
/**
*
*/
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();
} }
} }

View File

@ -83,7 +83,7 @@ public class KafkaPublisherTest {
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null); SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null);
publisher.publish(messageContext, fis, null); publisher.publish(messageContext, fis, null, 2000);
fis.close(); fis.close();
publisher.close(); publisher.close();
@ -105,9 +105,9 @@ public class KafkaPublisherTest {
Properties kafkaProperties = this.buildProducerProperties(); Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties); 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(); publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName); ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
@ -131,9 +131,9 @@ public class KafkaPublisherTest {
Properties kafkaProperties = this.buildProducerProperties(); Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties); 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(); publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName); ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
@ -157,10 +157,10 @@ public class KafkaPublisherTest {
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties); 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); messageContext.setFailedSegments(1, 3);
publisher.publish(messageContext, fis, null); publisher.publish(messageContext, fis, null, 2000);
publisher.close(); publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName); 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() { private Properties buildProducerProperties() {
Properties kafkaProperties = new Properties(); Properties kafkaProperties = new Properties();
kafkaProperties.setProperty("bootstrap.servers", "0.0.0.0:" + kafkaLocal.getKafkaPort()); kafkaProperties.setProperty("bootstrap.servers", "0.0.0.0:" + kafkaLocal.getKafkaPort());

View File

@ -32,9 +32,9 @@ public class SplittableMessageContextTest {
@Test @Test
public void validateFullSetting() { 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); 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("hello", new String(ctx.getKeyBytes(), StandardCharsets.UTF_8));
assertEquals("foo", ctx.getTopicName()); assertEquals("foo", ctx.getTopicName());
assertEquals("topic: 'foo'; delimiter: '\n'", ctx.toString()); assertEquals("topic: 'foo'; delimiter: '\n'", ctx.toString());
@ -44,7 +44,9 @@ public class SplittableMessageContextTest {
@Test @Test
public void validateToString() { public void validateToString() {
SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null); 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 @Test
@ -56,7 +58,7 @@ public class SplittableMessageContextTest {
ctx.setFailedSegmentsAsByteArray(null); ctx.setFailedSegmentsAsByteArray(null);
assertNull(ctx.getFailedSegments()); assertNull(ctx.getFailedSegments());
assertEquals("(\\W)\\Z", ctx.getDelimiterPattern());; assertNull(ctx.getDelimiterBytes());
assertNull(ctx.getKeyBytes()); assertNull(ctx.getKeyBytes());
assertNull(ctx.getKeyBytesAsString()); assertNull(ctx.getKeyBytesAsString());
assertEquals("foo", ctx.getTopicName()); assertEquals("foo", ctx.getTopicName());

View File

@ -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());
}
}

View File

@ -66,6 +66,7 @@ public class TestPutKafka {
} }
@Test @Test
@Ignore
public void testDelimitedMessagesWithKey() { public void testDelimitedMessagesWithKey() {
String topicName = "testDelimitedMessagesWithKey"; String topicName = "testDelimitedMessagesWithKey";
PutKafka putKafka = new PutKafka(); PutKafka putKafka = new PutKafka();
@ -95,7 +96,7 @@ public class TestPutKafka {
@Test @Test
@Ignore @Ignore
public void testWithFailureAndPartialResend() throws Exception { public void testWithFailureAndPartialResend() throws Exception {
String topicName = "testWithImmediateFailure"; String topicName = "testWithFailureAndPartialResend";
PutKafka putKafka = new PutKafka(); PutKafka putKafka = new PutKafka();
final TestRunner runner = TestRunners.newTestRunner(putKafka); final TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PutKafka.TOPIC, topicName); 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) { private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
Properties props = new Properties(); Properties props = new Properties();
props.put("zookeeper.connect", "0.0.0.0:" + kafkaLocal.getZookeeperPort()); props.put("zookeeper.connect", "0.0.0.0:" + kafkaLocal.getZookeeperPort());