mirror of https://github.com/apache/nifi.git
NIFI-271
This commit is contained in:
parent
b2a1f5217d
commit
8f2502c4e4
|
@ -61,97 +61,101 @@ import org.apache.nifi.processor.util.StandardValidators;
|
||||||
@SupportsBatching
|
@SupportsBatching
|
||||||
@CapabilityDescription("Fetches messages from Apache Kafka")
|
@CapabilityDescription("Fetches messages from Apache Kafka")
|
||||||
@Tags({"Kafka", "Apache", "Get", "Ingest", "Ingress", "Topic", "PubSub"})
|
@Tags({"Kafka", "Apache", "Get", "Ingest", "Ingress", "Topic", "PubSub"})
|
||||||
@WritesAttributes({ @WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"),
|
@WritesAttributes({
|
||||||
@WritesAttribute(attribute = "kafka.key", description = "The key of the Kafka message, if it exists and batch size is 1. If the message does not have a key, or if the batch size is greater than 1, this attribute will not be added"),
|
@WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"),
|
||||||
@WritesAttribute(attribute = "kafka.partition", description = "The partition of the Kafka Topic from which the message was received. This attribute is added only if the batch size is 1"),
|
@WritesAttribute(attribute = "kafka.key", description = "The key of the Kafka message, if it exists and batch size is 1. If"
|
||||||
@WritesAttribute(attribute = "kafka.offset", description = "The offset of the message within the Kafka partition. This attribute is added only if the batch size is 1") })
|
+ " the message does not have a key, or if the batch size is greater than 1, this attribute will not be added"),
|
||||||
|
@WritesAttribute(attribute = "kafka.partition", description = "The partition of the Kafka Topic from which the message was received. This attribute is added only if the batch size is 1"),
|
||||||
|
@WritesAttribute(attribute = "kafka.offset", description = "The offset of the message within the Kafka partition. This attribute is added only if the batch size is 1")})
|
||||||
public class GetKafka extends AbstractProcessor {
|
public class GetKafka extends AbstractProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
|
||||||
.name("ZooKeeper Connection String")
|
.name("ZooKeeper Connection String")
|
||||||
.description("The Connection String to use in order to connect to ZooKeeper. This is often a comma-separated list of <host>:<port> combinations. For example, host1:2181,host2:2181,host3:2188")
|
.description("The Connection String to use in order to connect to ZooKeeper. This is often a comma-separated list of <host>:<port>"
|
||||||
.required(true)
|
+ " combinations. For example, host1:2181,host2:2181,host3:2188")
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.required(true)
|
||||||
.expressionLanguageSupported(false)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.build();
|
.expressionLanguageSupported(false)
|
||||||
|
.build();
|
||||||
public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
||||||
.name("Topic Name")
|
.name("Topic Name")
|
||||||
.description("The Kafka Topic to pull messages from")
|
.description("The Kafka Topic to pull messages from")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor ZOOKEEPER_COMMIT_DELAY = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor ZOOKEEPER_COMMIT_DELAY = new PropertyDescriptor.Builder()
|
||||||
.name("Zookeeper Commit Frequency")
|
.name("Zookeeper Commit Frequency")
|
||||||
.description("Specifies how often to communicate with ZooKeeper to indicate which messages have been pulled. A longer time period will result in better overall performance but can result in more data duplication if a NiFi node is lost")
|
.description("Specifies how often to communicate with ZooKeeper to indicate which messages have been pulled. A longer time period will"
|
||||||
.required(true)
|
+ " result in better overall performance but can result in more data duplication if a NiFi node is lost")
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.required(true)
|
||||||
.expressionLanguageSupported(false)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.defaultValue("60 secs")
|
.expressionLanguageSupported(false)
|
||||||
.build();
|
.defaultValue("60 secs")
|
||||||
|
.build();
|
||||||
public static final PropertyDescriptor ZOOKEEPER_TIMEOUT = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor ZOOKEEPER_TIMEOUT = new PropertyDescriptor.Builder()
|
||||||
.name("ZooKeeper Communications Timeout")
|
.name("ZooKeeper Communications Timeout")
|
||||||
.description("The amount of time to wait for a response from ZooKeeper before determining that there is a communications error")
|
.description("The amount of time to wait for a response from ZooKeeper before determining that there is a communications error")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.defaultValue("30 secs")
|
.defaultValue("30 secs")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor KAFKA_TIMEOUT = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor KAFKA_TIMEOUT = new PropertyDescriptor.Builder()
|
||||||
.name("Kafka Communications Timeout")
|
.name("Kafka Communications Timeout")
|
||||||
.description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
|
.description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.defaultValue("30 secs")
|
.defaultValue("30 secs")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||||
.name("Batch Size")
|
.name("Batch Size")
|
||||||
.description("Specifies the maximum number of messages to combine into a single FlowFile. These messages will be "
|
.description("Specifies the maximum number of messages to combine into a single FlowFile. These messages will be "
|
||||||
+ "concatenated together with the <Message Demarcator> string placed between the content of each message. "
|
+ "concatenated together with the <Message Demarcator> string placed between the content of each message. "
|
||||||
+ "If the messages from Kafka should not be concatenated together, leave this value at 1.")
|
+ "If the messages from Kafka should not be concatenated together, leave this value at 1.")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.defaultValue("1")
|
.defaultValue("1")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
|
||||||
.name("Message Demarcator")
|
.name("Message Demarcator")
|
||||||
.description("Specifies the characters to use in order to demarcate multiple messages from Kafka. If the <Batch Size> "
|
.description("Specifies the characters to use in order to demarcate multiple messages from Kafka. If the <Batch Size> "
|
||||||
+ "property is set to 1, this value is ignored. Otherwise, for each two subsequent messages in the batch, "
|
+ "property is set to 1, this value is ignored. Otherwise, for each two subsequent messages in the batch, "
|
||||||
+ "this value will be placed in between them.")
|
+ "this value will be placed in between them.")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(Validator.VALID) // accept anything as a demarcator, including empty string
|
.addValidator(Validator.VALID) // accept anything as a demarcator, including empty string
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.defaultValue("\\n")
|
.defaultValue("\\n")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder()
|
||||||
.name("Client Name")
|
.name("Client Name")
|
||||||
.description("Client Name to use when communicating with Kafka")
|
.description("Client Name to use when communicating with Kafka")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||||
.name("success")
|
.name("success")
|
||||||
.description("All FlowFiles that are created are routed to this relationship")
|
.description("All FlowFiles that are created are routed to this relationship")
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
|
||||||
private final BlockingQueue<ConsumerIterator<byte[], byte[]>> streamIterators = new LinkedBlockingQueue<>();
|
private final BlockingQueue<ConsumerIterator<byte[], byte[]>> streamIterators = new LinkedBlockingQueue<>();
|
||||||
private volatile ConsumerConnector consumer;
|
private volatile ConsumerConnector consumer;
|
||||||
|
|
||||||
final Lock interruptionLock = new ReentrantLock();
|
final Lock interruptionLock = new ReentrantLock();
|
||||||
// guarded by interruptionLock
|
// guarded by interruptionLock
|
||||||
private final Set<Thread> interruptableThreads = new HashSet<>();
|
private final Set<Thread> interruptableThreads = new HashSet<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder()
|
final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder()
|
||||||
.fromPropertyDescriptor(CLIENT_NAME)
|
.fromPropertyDescriptor(CLIENT_NAME)
|
||||||
.defaultValue("NiFi-" + getIdentifier())
|
.defaultValue("NiFi-" + getIdentifier())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||||
props.add(ZOOKEEPER_CONNECTION_STRING);
|
props.add(ZOOKEEPER_CONNECTION_STRING);
|
||||||
props.add(TOPIC);
|
props.add(TOPIC);
|
||||||
|
@ -163,174 +167,174 @@ public class GetKafka extends AbstractProcessor {
|
||||||
props.add(ZOOKEEPER_TIMEOUT);
|
props.add(ZOOKEEPER_TIMEOUT);
|
||||||
return props;
|
return props;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<Relationship> getRelationships() {
|
public Set<Relationship> getRelationships() {
|
||||||
final Set<Relationship> relationships = new HashSet<>(1);
|
final Set<Relationship> relationships = new HashSet<>(1);
|
||||||
relationships.add(REL_SUCCESS);
|
relationships.add(REL_SUCCESS);
|
||||||
return relationships;
|
return relationships;
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnScheduled
|
@OnScheduled
|
||||||
public void createConsumers(final ProcessContext context) {
|
public void createConsumers(final ProcessContext context) {
|
||||||
final String topic = context.getProperty(TOPIC).getValue();
|
final String topic = context.getProperty(TOPIC).getValue();
|
||||||
|
|
||||||
final Map<String, Integer> topicCountMap = new HashMap<>(1);
|
final Map<String, Integer> topicCountMap = new HashMap<>(1);
|
||||||
topicCountMap.put(topic, context.getMaxConcurrentTasks());
|
topicCountMap.put(topic, context.getMaxConcurrentTasks());
|
||||||
|
|
||||||
final Properties props = new Properties();
|
final Properties props = new Properties();
|
||||||
props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue());
|
props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue());
|
||||||
props.setProperty("group.id", getIdentifier());
|
props.setProperty("group.id", getIdentifier());
|
||||||
props.setProperty("auto.commit.interval.ms", String.valueOf(context.getProperty(ZOOKEEPER_COMMIT_DELAY).asTimePeriod(TimeUnit.MILLISECONDS)));
|
props.setProperty("auto.commit.interval.ms", String.valueOf(context.getProperty(ZOOKEEPER_COMMIT_DELAY).asTimePeriod(TimeUnit.MILLISECONDS)));
|
||||||
props.setProperty("auto.commit.enable", "true"); // just be explicit
|
props.setProperty("auto.commit.enable", "true"); // just be explicit
|
||||||
props.setProperty("auto.offset.reset", "smallest");
|
props.setProperty("auto.offset.reset", "smallest");
|
||||||
|
|
||||||
final ConsumerConfig consumerConfig = new ConsumerConfig(props);
|
final ConsumerConfig consumerConfig = new ConsumerConfig(props);
|
||||||
consumer = Consumer.createJavaConsumerConnector(consumerConfig);
|
consumer = Consumer.createJavaConsumerConnector(consumerConfig);
|
||||||
|
|
||||||
final Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
|
final Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
|
||||||
final List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
|
final List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
|
||||||
|
|
||||||
this.streamIterators.clear();
|
this.streamIterators.clear();
|
||||||
|
|
||||||
for ( final KafkaStream<byte[], byte[]> stream : streams ) {
|
for (final KafkaStream<byte[], byte[]> stream : streams) {
|
||||||
streamIterators.add(stream.iterator());
|
streamIterators.add(stream.iterator());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnStopped
|
@OnStopped
|
||||||
public void shutdownConsumer() {
|
public void shutdownConsumer() {
|
||||||
if ( consumer != null ) {
|
if (consumer != null) {
|
||||||
try {
|
try {
|
||||||
consumer.commitOffsets();
|
consumer.commitOffsets();
|
||||||
} finally {
|
} finally {
|
||||||
consumer.shutdown();
|
consumer.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnUnscheduled
|
@OnUnscheduled
|
||||||
public void interruptIterators() {
|
public void interruptIterators() {
|
||||||
// Kafka doesn't provide a non-blocking API for pulling messages. We can, however,
|
// Kafka doesn't provide a non-blocking API for pulling messages. We can, however,
|
||||||
// interrupt the Threads. We do this when the Processor is stopped so that we have the
|
// interrupt the Threads. We do this when the Processor is stopped so that we have the
|
||||||
// ability to shutdown the Processor.
|
// ability to shutdown the Processor.
|
||||||
interruptionLock.lock();
|
interruptionLock.lock();
|
||||||
try {
|
try {
|
||||||
for ( final Thread t : interruptableThreads ) {
|
for (final Thread t : interruptableThreads) {
|
||||||
t.interrupt();
|
t.interrupt();
|
||||||
}
|
}
|
||||||
|
|
||||||
interruptableThreads.clear();
|
interruptableThreads.clear();
|
||||||
} finally {
|
} finally {
|
||||||
interruptionLock.unlock();
|
interruptionLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
|
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
|
||||||
return streamIterators.poll();
|
return streamIterators.poll();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||||
ConsumerIterator<byte[], byte[]> iterator = getStreamIterator();
|
ConsumerIterator<byte[], byte[]> iterator = getStreamIterator();
|
||||||
if ( iterator == null ) {
|
if (iterator == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
|
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
|
||||||
final String demarcator = context.getProperty(MESSAGE_DEMARCATOR).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
final String demarcator = context.getProperty(MESSAGE_DEMARCATOR).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||||
final byte[] demarcatorBytes = demarcator.getBytes(StandardCharsets.UTF_8);
|
final byte[] demarcatorBytes = demarcator.getBytes(StandardCharsets.UTF_8);
|
||||||
final String topic = context.getProperty(TOPIC).getValue();
|
final String topic = context.getProperty(TOPIC).getValue();
|
||||||
|
|
||||||
FlowFile flowFile = null;
|
FlowFile flowFile = null;
|
||||||
try {
|
try {
|
||||||
// add the current thread to the Set of those to be interrupted if processor stopped.
|
// add the current thread to the Set of those to be interrupted if processor stopped.
|
||||||
interruptionLock.lock();
|
interruptionLock.lock();
|
||||||
try {
|
try {
|
||||||
interruptableThreads.add(Thread.currentThread());
|
interruptableThreads.add(Thread.currentThread());
|
||||||
} finally {
|
} finally {
|
||||||
interruptionLock.unlock();
|
interruptionLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
final long start = System.nanoTime();
|
final long start = System.nanoTime();
|
||||||
flowFile = session.create();
|
flowFile = session.create();
|
||||||
|
|
||||||
final Map<String, String> attributes = new HashMap<>();
|
final Map<String, String> attributes = new HashMap<>();
|
||||||
attributes.put("kafka.topic", topic);
|
attributes.put("kafka.topic", topic);
|
||||||
|
|
||||||
int numMessages = 0;
|
int numMessages = 0;
|
||||||
for (int msgCount = 0; msgCount < batchSize; msgCount++) {
|
for (int msgCount = 0; msgCount < batchSize; msgCount++) {
|
||||||
// if the processor is stopped, iterator.hasNext() will throw an Exception.
|
// if the processor is stopped, iterator.hasNext() will throw an Exception.
|
||||||
// In this case, we just break out of the loop.
|
// In this case, we just break out of the loop.
|
||||||
try {
|
try {
|
||||||
if ( !iterator.hasNext() ) {
|
if (!iterator.hasNext()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
final MessageAndMetadata<byte[], byte[]> mam = iterator.next();
|
final MessageAndMetadata<byte[], byte[]> mam = iterator.next();
|
||||||
if ( mam == null ) {
|
if (mam == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final byte[] key = mam.key();
|
final byte[] key = mam.key();
|
||||||
|
|
||||||
if ( batchSize == 1 ) {
|
if (batchSize == 1) {
|
||||||
// the kafka.key, kafka.offset, and kafka.partition attributes are added only
|
// the kafka.key, kafka.offset, and kafka.partition attributes are added only
|
||||||
// for a batch size of 1.
|
// for a batch size of 1.
|
||||||
if ( key != null ) {
|
if (key != null) {
|
||||||
attributes.put("kafka.key", new String(key, StandardCharsets.UTF_8));
|
attributes.put("kafka.key", new String(key, StandardCharsets.UTF_8));
|
||||||
}
|
}
|
||||||
|
|
||||||
attributes.put("kafka.offset", String.valueOf(mam.offset()));
|
attributes.put("kafka.offset", String.valueOf(mam.offset()));
|
||||||
attributes.put("kafka.partition", String.valueOf(mam.partition()));
|
attributes.put("kafka.partition", String.valueOf(mam.partition()));
|
||||||
}
|
}
|
||||||
|
|
||||||
// add the message to the FlowFile's contents
|
// add the message to the FlowFile's contents
|
||||||
final boolean firstMessage = (msgCount == 0);
|
final boolean firstMessage = (msgCount == 0);
|
||||||
flowFile = session.append(flowFile, new OutputStreamCallback() {
|
flowFile = session.append(flowFile, new OutputStreamCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void process(final OutputStream out) throws IOException {
|
public void process(final OutputStream out) throws IOException {
|
||||||
if ( !firstMessage ) {
|
if (!firstMessage) {
|
||||||
out.write(demarcatorBytes);
|
out.write(demarcatorBytes);
|
||||||
}
|
}
|
||||||
out.write(mam.message());
|
out.write(mam.message());
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
numMessages++;
|
numMessages++;
|
||||||
}
|
}
|
||||||
|
|
||||||
// If we received no messages, remove the FlowFile. Otherwise, send to success.
|
// If we received no messages, remove the FlowFile. Otherwise, send to success.
|
||||||
if ( flowFile.getSize() == 0L ) {
|
if (flowFile.getSize() == 0L) {
|
||||||
session.remove(flowFile);
|
session.remove(flowFile);
|
||||||
} else {
|
} else {
|
||||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||||
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
||||||
session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, "Received " + numMessages + " Kafka messages", millis);
|
session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, "Received " + numMessages + " Kafka messages", millis);
|
||||||
getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[] {flowFile, numMessages, millis});
|
getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[]{flowFile, numMessages, millis});
|
||||||
session.transfer(flowFile, REL_SUCCESS);
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
}
|
}
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
getLogger().error("Failed to receive FlowFile from Kafka due to {}", new Object[] {e});
|
getLogger().error("Failed to receive FlowFile from Kafka due to {}", new Object[]{e});
|
||||||
if ( flowFile != null ) {
|
if (flowFile != null) {
|
||||||
session.remove(flowFile);
|
session.remove(flowFile);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// Remove the current thread from the Set of Threads to interrupt.
|
// Remove the current thread from the Set of Threads to interrupt.
|
||||||
interruptionLock.lock();
|
interruptionLock.lock();
|
||||||
try {
|
try {
|
||||||
interruptableThreads.remove(Thread.currentThread());
|
interruptableThreads.remove(Thread.currentThread());
|
||||||
} finally {
|
} finally {
|
||||||
interruptionLock.unlock();
|
interruptionLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add the iterator back to the queue
|
// Add the iterator back to the queue
|
||||||
if ( iterator != null ) {
|
if (iterator != null) {
|
||||||
streamIterators.offer(iterator);
|
streamIterators.offer(iterator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -61,95 +61,100 @@ import scala.actors.threadpool.Arrays;
|
||||||
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub"})
|
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub"})
|
||||||
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka")
|
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka")
|
||||||
public class PutKafka extends AbstractProcessor {
|
public class PutKafka extends AbstractProcessor {
|
||||||
|
|
||||||
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
|
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
|
||||||
private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*";
|
private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*";
|
||||||
|
|
||||||
public static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("-1", "Guarantee Replicated Delivery", "FlowFile will be routed to failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration");
|
|
||||||
public static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery", "FlowFile will be routed to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than <Guarantee Replicated Delivery> but can result in data loss if a Kafka node crashes");
|
|
||||||
public static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort", "FlowFile will be routed to success after successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result in data loss.");
|
|
||||||
|
|
||||||
public static final PropertyDescriptor SEED_BROKERS = new PropertyDescriptor.Builder()
|
|
||||||
.name("Known Brokers")
|
|
||||||
.description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
|
|
||||||
.required(true)
|
|
||||||
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX)))
|
|
||||||
.expressionLanguageSupported(false)
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
|
||||||
.name("Topic Name")
|
|
||||||
.description("The Kafka Topic of interest")
|
|
||||||
.required(true)
|
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(true)
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
|
|
||||||
.name("Kafka Key")
|
|
||||||
.description("The Key to use for the Message")
|
|
||||||
.required(false)
|
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(true)
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
|
|
||||||
.name("Delivery Guarantee")
|
|
||||||
.description("Specifies the requirement for guaranteeing that a message is sent to Kafka")
|
|
||||||
.required(true)
|
|
||||||
.expressionLanguageSupported(false)
|
|
||||||
.allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
|
|
||||||
.defaultValue(DELIVERY_BEST_EFFORT.getValue())
|
|
||||||
.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. "
|
|
||||||
+ "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.")
|
|
||||||
.required(false)
|
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(true)
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
|
||||||
.name("Max Buffer Size")
|
|
||||||
.description("The maximum amount of data to buffer in memory before sending to Kafka")
|
|
||||||
.required(true)
|
|
||||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(false)
|
|
||||||
.defaultValue("1 MB")
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
|
|
||||||
.name("Communications Timeout")
|
|
||||||
.description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
|
|
||||||
.required(true)
|
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(false)
|
|
||||||
.defaultValue("30 secs")
|
|
||||||
.build();
|
|
||||||
public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder()
|
|
||||||
.name("Client Name")
|
|
||||||
.description("Client Name to use when communicating with Kafka")
|
|
||||||
.required(true)
|
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
|
||||||
.expressionLanguageSupported(false)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
|
public static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("-1", "Guarantee Replicated Delivery", "FlowFile will be routed to"
|
||||||
|
+ " failure unless the message is replicated to the appropriate number of Kafka Nodes according to the Topic configuration");
|
||||||
|
public static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery", "FlowFile will be routed"
|
||||||
|
+ " to success if the message is received by a single Kafka node, whether or not it is replicated. This is faster than"
|
||||||
|
+ " <Guarantee Replicated Delivery> but can result in data loss if a Kafka node crashes");
|
||||||
|
public static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort", "FlowFile will be routed to success after"
|
||||||
|
+ " successfully writing the content to a Kafka node, without waiting for a response. This provides the best performance but may result"
|
||||||
|
+ " in data loss.");
|
||||||
|
|
||||||
|
public static final PropertyDescriptor SEED_BROKERS = new PropertyDescriptor.Builder()
|
||||||
|
.name("Known Brokers")
|
||||||
|
.description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX)))
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
||||||
|
.name("Topic Name")
|
||||||
|
.description("The Kafka Topic of interest")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
|
||||||
|
.name("Kafka Key")
|
||||||
|
.description("The Key to use for the Message")
|
||||||
|
.required(false)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Delivery Guarantee")
|
||||||
|
.description("Specifies the requirement for guaranteeing that a message is sent to Kafka")
|
||||||
|
.required(true)
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
|
||||||
|
.defaultValue(DELIVERY_BEST_EFFORT.getValue())
|
||||||
|
.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. "
|
||||||
|
+ "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.")
|
||||||
|
.required(false)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Max Buffer Size")
|
||||||
|
.description("The maximum amount of data to buffer in memory before sending to Kafka")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.defaultValue("1 MB")
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
|
||||||
|
.name("Communications Timeout")
|
||||||
|
.description("The amount of time to wait for a response from Kafka before determining that there is a communications error")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.defaultValue("30 secs")
|
||||||
|
.build();
|
||||||
|
public static final PropertyDescriptor CLIENT_NAME = new PropertyDescriptor.Builder()
|
||||||
|
.name("Client Name")
|
||||||
|
.description("Client Name to use when communicating with Kafka")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.build();
|
||||||
|
|
||||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||||
.name("success")
|
.name("success")
|
||||||
.description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship")
|
.description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship")
|
||||||
.build();
|
.build();
|
||||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||||
.name("failure")
|
.name("failure")
|
||||||
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
private final BlockingQueue<Producer<byte[], byte[]>> producers = new LinkedBlockingQueue<>();
|
private final BlockingQueue<Producer<byte[], byte[]>> producers = new LinkedBlockingQueue<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
final PropertyDescriptor clientName = new PropertyDescriptor.Builder()
|
final PropertyDescriptor clientName = new PropertyDescriptor.Builder()
|
||||||
.fromPropertyDescriptor(CLIENT_NAME)
|
.fromPropertyDescriptor(CLIENT_NAME)
|
||||||
.defaultValue("NiFi-" + getIdentifier())
|
.defaultValue("NiFi-" + getIdentifier())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||||
props.add(SEED_BROKERS);
|
props.add(SEED_BROKERS);
|
||||||
props.add(TOPIC);
|
props.add(TOPIC);
|
||||||
|
@ -161,7 +166,7 @@ public class PutKafka extends AbstractProcessor {
|
||||||
props.add(clientName);
|
props.add(clientName);
|
||||||
return props;
|
return props;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<Relationship> getRelationships() {
|
public Set<Relationship> getRelationships() {
|
||||||
final Set<Relationship> relationships = new HashSet<>(1);
|
final Set<Relationship> relationships = new HashSet<>(1);
|
||||||
|
@ -169,17 +174,16 @@ public class PutKafka extends AbstractProcessor {
|
||||||
relationships.add(REL_FAILURE);
|
relationships.add(REL_FAILURE);
|
||||||
return relationships;
|
return relationships;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@OnStopped
|
@OnStopped
|
||||||
public void closeProducers() {
|
public void closeProducers() {
|
||||||
Producer<byte[], byte[]> producer;
|
Producer<byte[], byte[]> producer;
|
||||||
|
|
||||||
while ((producer = producers.poll()) != null) {
|
while ((producer = producers.poll()) != null) {
|
||||||
producer.close();
|
producer.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ProducerConfig createConfig(final ProcessContext context) {
|
protected ProducerConfig createConfig(final ProcessContext context) {
|
||||||
final String brokers = context.getProperty(SEED_BROKERS).getValue();
|
final String brokers = context.getProperty(SEED_BROKERS).getValue();
|
||||||
|
|
||||||
|
@ -188,76 +192,76 @@ public class PutKafka extends AbstractProcessor {
|
||||||
properties.setProperty("request.required.acks", context.getProperty(DELIVERY_GUARANTEE).getValue());
|
properties.setProperty("request.required.acks", context.getProperty(DELIVERY_GUARANTEE).getValue());
|
||||||
properties.setProperty("client.id", context.getProperty(CLIENT_NAME).getValue());
|
properties.setProperty("client.id", context.getProperty(CLIENT_NAME).getValue());
|
||||||
properties.setProperty("request.timeout.ms", String.valueOf(context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).longValue()));
|
properties.setProperty("request.timeout.ms", String.valueOf(context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).longValue()));
|
||||||
|
|
||||||
properties.setProperty("message.send.max.retries", "1");
|
properties.setProperty("message.send.max.retries", "1");
|
||||||
properties.setProperty("producer.type", "sync");
|
properties.setProperty("producer.type", "sync");
|
||||||
|
|
||||||
return new ProducerConfig(properties);
|
return new ProducerConfig(properties);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
||||||
return new Producer<>(createConfig(context));
|
return new Producer<>(createConfig(context));
|
||||||
}
|
}
|
||||||
|
|
||||||
private Producer<byte[], byte[]> borrowProducer(final ProcessContext context) {
|
private Producer<byte[], byte[]> borrowProducer(final ProcessContext context) {
|
||||||
Producer<byte[], byte[]> producer = producers.poll();
|
Producer<byte[], byte[]> producer = producers.poll();
|
||||||
return producer == null ? createProducer(context) : producer;
|
return producer == null ? createProducer(context) : producer;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void returnProducer(final Producer<byte[], byte[]> producer) {
|
private void returnProducer(final Producer<byte[], byte[]> producer) {
|
||||||
producers.offer(producer);
|
producers.offer(producer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||||
FlowFile flowFile = session.get();
|
FlowFile flowFile = session.get();
|
||||||
if ( flowFile == null ) {
|
if (flowFile == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final long start = System.nanoTime();
|
final long start = System.nanoTime();
|
||||||
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
final byte[] keyBytes = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8);
|
final byte[] keyBytes = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8);
|
||||||
String delimiter = context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue();
|
String delimiter = context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
if ( delimiter != null ) {
|
if (delimiter != null) {
|
||||||
delimiter = delimiter.replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
delimiter = delimiter.replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||||
}
|
}
|
||||||
|
|
||||||
final long maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue();
|
final long maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue();
|
||||||
final Producer<byte[], byte[]> producer = borrowProducer(context);
|
final Producer<byte[], byte[]> producer = borrowProducer(context);
|
||||||
|
|
||||||
if ( delimiter == null ) {
|
if (delimiter == null) {
|
||||||
// Send the entire FlowFile as a single message.
|
// Send the entire FlowFile as a single message.
|
||||||
final byte[] value = new byte[(int) flowFile.getSize()];
|
final byte[] value = new byte[(int) flowFile.getSize()];
|
||||||
session.read(flowFile, new InputStreamCallback() {
|
session.read(flowFile, new InputStreamCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void process(final InputStream in) throws IOException {
|
public void process(final InputStream in) throws IOException {
|
||||||
StreamUtils.fillBuffer(in, value);
|
StreamUtils.fillBuffer(in, value);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
boolean error = false;
|
boolean error = false;
|
||||||
try {
|
try {
|
||||||
final KeyedMessage<byte[], byte[]> message;
|
final KeyedMessage<byte[], byte[]> message;
|
||||||
if ( key == null ) {
|
if (key == null) {
|
||||||
message = new KeyedMessage<>(topic, value);
|
message = new KeyedMessage<>(topic, value);
|
||||||
} else {
|
} else {
|
||||||
message = new KeyedMessage<>(topic, keyBytes, value);
|
message = new KeyedMessage<>(topic, keyBytes, value);
|
||||||
}
|
}
|
||||||
|
|
||||||
producer.send(message);
|
producer.send(message);
|
||||||
final long nanos = System.nanoTime() - start;
|
final long nanos = System.nanoTime() - start;
|
||||||
|
|
||||||
session.getProvenanceReporter().send(flowFile, "kafka://" + topic);
|
session.getProvenanceReporter().send(flowFile, "kafka://" + topic);
|
||||||
session.transfer(flowFile, REL_SUCCESS);
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[] {flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
|
getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[]{flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
getLogger().error("Failed to send {} to Kafka due to {}; routing to failure", new Object[] {flowFile, e});
|
getLogger().error("Failed to send {} to Kafka due to {}; routing to failure", new Object[]{flowFile, e});
|
||||||
session.transfer(flowFile, REL_FAILURE);
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
error = true;
|
error = true;
|
||||||
} finally {
|
} finally {
|
||||||
if ( error ) {
|
if (error) {
|
||||||
producer.close();
|
producer.close();
|
||||||
} else {
|
} else {
|
||||||
returnProducer(producer);
|
returnProducer(producer);
|
||||||
|
@ -265,53 +269,53 @@ public class PutKafka extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
final byte[] delimiterBytes = delimiter.getBytes(StandardCharsets.UTF_8);
|
final byte[] delimiterBytes = delimiter.getBytes(StandardCharsets.UTF_8);
|
||||||
|
|
||||||
// The NonThreadSafeCircularBuffer allows us to add a byte from the stream one at a time and see
|
// The NonThreadSafeCircularBuffer allows us to add a byte from the stream one at a time and see
|
||||||
// if it matches some pattern. We can use this to search for the delimiter as we read through
|
// if it matches some pattern. We can use this to search for the delimiter as we read through
|
||||||
// the stream of bytes in the FlowFile
|
// the stream of bytes in the FlowFile
|
||||||
final NonThreadSafeCircularBuffer buffer = new NonThreadSafeCircularBuffer(delimiterBytes);
|
final NonThreadSafeCircularBuffer buffer = new NonThreadSafeCircularBuffer(delimiterBytes);
|
||||||
|
|
||||||
boolean error = false;
|
boolean error = false;
|
||||||
final LongHolder lastMessageOffset = new LongHolder(0L);
|
final LongHolder lastMessageOffset = new LongHolder(0L);
|
||||||
final LongHolder messagesSent = new LongHolder(0L);
|
final LongHolder messagesSent = new LongHolder(0L);
|
||||||
|
|
||||||
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
|
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
|
||||||
session.read(flowFile, new InputStreamCallback() {
|
session.read(flowFile, new InputStreamCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void process(final InputStream rawIn) throws IOException {
|
public void process(final InputStream rawIn) throws IOException {
|
||||||
byte[] data = null; // contents of a single message
|
byte[] data = null; // contents of a single message
|
||||||
|
|
||||||
boolean streamFinished = false;
|
boolean streamFinished = false;
|
||||||
|
|
||||||
final List<KeyedMessage<byte[], byte[]>> messages = new ArrayList<>(); // batch to send
|
final List<KeyedMessage<byte[], byte[]>> messages = new ArrayList<>(); // batch to send
|
||||||
long messageBytes = 0L; // size of messages in the 'messages' list
|
long messageBytes = 0L; // size of messages in the 'messages' list
|
||||||
|
|
||||||
int nextByte;
|
int nextByte;
|
||||||
try (final InputStream bufferedIn = new BufferedInputStream(rawIn);
|
try (final InputStream bufferedIn = new BufferedInputStream(rawIn);
|
||||||
final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) {
|
final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) {
|
||||||
|
|
||||||
// read until we're out of data.
|
// read until we're out of data.
|
||||||
while (!streamFinished) {
|
while (!streamFinished) {
|
||||||
nextByte = in.read();
|
nextByte = in.read();
|
||||||
|
|
||||||
if ( nextByte > -1 ) {
|
if (nextByte > -1) {
|
||||||
baos.write(nextByte);
|
baos.write(nextByte);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (nextByte == -1) {
|
if (nextByte == -1) {
|
||||||
// we ran out of data. This message is complete.
|
// we ran out of data. This message is complete.
|
||||||
data = baos.toByteArray();
|
data = baos.toByteArray();
|
||||||
streamFinished = true;
|
streamFinished = true;
|
||||||
} else if ( buffer.addAndCompare((byte) nextByte) ) {
|
} else if (buffer.addAndCompare((byte) nextByte)) {
|
||||||
// we matched our delimiter. This message is complete. We want all of the bytes from the
|
// we matched our delimiter. This message is complete. We want all of the bytes from the
|
||||||
// underlying BAOS exception for the last 'delimiterBytes.length' bytes because we don't want
|
// underlying BAOS exception for the last 'delimiterBytes.length' bytes because we don't want
|
||||||
// the delimiter itself to be sent.
|
// the delimiter itself to be sent.
|
||||||
data = Arrays.copyOfRange(baos.getUnderlyingBuffer(), 0, baos.size() - delimiterBytes.length);
|
data = Arrays.copyOfRange(baos.getUnderlyingBuffer(), 0, baos.size() - delimiterBytes.length);
|
||||||
}
|
}
|
||||||
|
|
||||||
if ( data != null ) {
|
if (data != null) {
|
||||||
// If the message has no data, ignore it.
|
// If the message has no data, ignore it.
|
||||||
if ( data.length != 0 ) {
|
if (data.length != 0) {
|
||||||
// either we ran out of data or we reached the end of the message.
|
// either we ran out of data or we reached the end of the message.
|
||||||
// Either way, create the message because it's ready to send.
|
// Either way, create the message because it's ready to send.
|
||||||
final KeyedMessage<byte[], byte[]> message;
|
final KeyedMessage<byte[], byte[]> message;
|
||||||
|
@ -361,7 +365,7 @@ public class PutKafka extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
// If there are messages left, send them
|
// If there are messages left, send them
|
||||||
if ( !messages.isEmpty() ) {
|
if (!messages.isEmpty()) {
|
||||||
try {
|
try {
|
||||||
messagesSent.addAndGet(messages.size()); // add count of messages
|
messagesSent.addAndGet(messages.size()); // add count of messages
|
||||||
producer.send(messages);
|
producer.send(messages);
|
||||||
|
@ -372,44 +376,45 @@ public class PutKafka extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
final long nanos = System.nanoTime() - start;
|
final long nanos = System.nanoTime() - start;
|
||||||
session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages");
|
session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages");
|
||||||
session.transfer(flowFile, REL_SUCCESS);
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[] {messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
|
getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[]{messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
|
||||||
} catch (final ProcessException pe) {
|
} catch (final ProcessException pe) {
|
||||||
error = true;
|
error = true;
|
||||||
|
|
||||||
// There was a failure sending messages to Kafka. Iff the lastMessageOffset is 0, then all of them failed and we can
|
// There was a failure sending messages to Kafka. Iff the lastMessageOffset is 0, then all of them failed and we can
|
||||||
// just route the FlowFile to failure. Otherwise, some messages were successful, so split them off and send them to
|
// just route the FlowFile to failure. Otherwise, some messages were successful, so split them off and send them to
|
||||||
// 'success' while we send the others to 'failure'.
|
// 'success' while we send the others to 'failure'.
|
||||||
final long offset = lastMessageOffset.get();
|
final long offset = lastMessageOffset.get();
|
||||||
if ( offset == 0L ) {
|
if (offset == 0L) {
|
||||||
// all of the messages failed to send. Route FlowFile to failure
|
// all of the messages failed to send. Route FlowFile to failure
|
||||||
getLogger().error("Failed to send {} to Kafka due to {}; routing to fialure", new Object[] {flowFile, pe.getCause()});
|
getLogger().error("Failed to send {} to Kafka due to {}; routing to fialure", new Object[]{flowFile, pe.getCause()});
|
||||||
session.transfer(flowFile, REL_FAILURE);
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
} else {
|
} else {
|
||||||
// Some of the messages were sent successfully. We want to split off the successful messages from the failed messages.
|
// Some of the messages were sent successfully. We want to split off the successful messages from the failed messages.
|
||||||
final FlowFile successfulMessages = session.clone(flowFile, 0L, offset);
|
final FlowFile successfulMessages = session.clone(flowFile, 0L, offset);
|
||||||
final FlowFile failedMessages = session.clone(flowFile, offset, flowFile.getSize() - offset);
|
final FlowFile failedMessages = session.clone(flowFile, offset, flowFile.getSize() - offset);
|
||||||
|
|
||||||
getLogger().error("Successfully sent {} of the messages from {} but then failed to send the rest. Original FlowFile split into two: {} routed to 'success', {} routed to 'failure'. Failure was due to {}", new Object[] {
|
getLogger().error("Successfully sent {} of the messages from {} but then failed to send the rest. Original FlowFile split into"
|
||||||
messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause() });
|
+ " two: {} routed to 'success', {} routed to 'failure'. Failure was due to {}", new Object[]{
|
||||||
|
messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause()});
|
||||||
|
|
||||||
session.transfer(successfulMessages, REL_SUCCESS);
|
session.transfer(successfulMessages, REL_SUCCESS);
|
||||||
session.transfer(failedMessages, REL_FAILURE);
|
session.transfer(failedMessages, REL_FAILURE);
|
||||||
session.remove(flowFile);
|
session.remove(flowFile);
|
||||||
session.getProvenanceReporter().send(successfulMessages, "kafka://" + topic);
|
session.getProvenanceReporter().send(successfulMessages, "kafka://" + topic);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
if ( error ) {
|
if (error) {
|
||||||
producer.close();
|
producer.close();
|
||||||
} else {
|
} else {
|
||||||
returnProducer(producer);
|
returnProducer(producer);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,22 +24,22 @@
|
||||||
<!-- Processor Documentation ================================================== -->
|
<!-- Processor Documentation ================================================== -->
|
||||||
<h2>Description:</h2>
|
<h2>Description:</h2>
|
||||||
<p>
|
<p>
|
||||||
This Processors polls <a href="http://kafka.apache.org/">Apache Kafka</a>
|
This Processors polls <a href="http://kafka.apache.org/">Apache Kafka</a>
|
||||||
for data. When a message is received from Kafka, this Processor emits a FlowFile
|
for data. When a message is received from Kafka, this Processor emits a FlowFile
|
||||||
where the content of the FlowFile is the value of the Kafka message. If the
|
where the content of the FlowFile is the value of the Kafka message. If the
|
||||||
message has a key associated with it, an attribute named <code>kafka.key</code>
|
message has a key associated with it, an attribute named <code>kafka.key</code>
|
||||||
will be added to the FlowFile, with the value being the UTF-8 Encoded value
|
will be added to the FlowFile, with the value being the UTF-8 Encoded value
|
||||||
of the Message's Key.
|
of the Message's Key.
|
||||||
</p>
|
</p>
|
||||||
<p>
|
<p>
|
||||||
Kafka supports the notion of a Consumer Group when pulling messages in order to
|
Kafka supports the notion of a Consumer Group when pulling messages in order to
|
||||||
provide scalability while still offering a publish-subscribe interface. Each
|
provide scalability while still offering a publish-subscribe interface. Each
|
||||||
Consumer Group must have a unique identifier. The Consumer Group identifier that
|
Consumer Group must have a unique identifier. The Consumer Group identifier that
|
||||||
is used by NiFi is the UUID of the Processor. This means that all of the nodes
|
is used by NiFi is the UUID of the Processor. This means that all of the nodes
|
||||||
within a cluster will use the same Consumer Group Identifier so that they do
|
within a cluster will use the same Consumer Group Identifier so that they do
|
||||||
not receive duplicate data but multiple GetKafka Processors can be used to pull
|
not receive duplicate data but multiple GetKafka Processors can be used to pull
|
||||||
from multiple Topics, as each Processor will receive a different Processor UUID
|
from multiple Topics, as each Processor will receive a different Processor UUID
|
||||||
and therefore a different Consumer Group Identifier.
|
and therefore a different Consumer Group Identifier.
|
||||||
</p>
|
</p>
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
|
|
@ -24,22 +24,22 @@
|
||||||
<!-- Processor Documentation ================================================== -->
|
<!-- Processor Documentation ================================================== -->
|
||||||
<h2>Description:</h2>
|
<h2>Description:</h2>
|
||||||
<p>
|
<p>
|
||||||
This Processors puts the contents of a FlowFile to a Topic in
|
This Processors puts the contents of a FlowFile to a Topic in
|
||||||
<a href="http://kafka.apache.org/">Apache Kafka</a>. The full contents of
|
<a href="http://kafka.apache.org/">Apache Kafka</a>. The full contents of
|
||||||
a FlowFile becomes the contents of a single message in Kafka.
|
a FlowFile becomes the contents of a single message in Kafka.
|
||||||
This message is optionally assigned a key by using the
|
This message is optionally assigned a key by using the
|
||||||
<Kafka Key> Property.
|
<Kafka Key> Property.
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<p>
|
<p>
|
||||||
The Processor allows the user to configure an optional Message Delimiter that
|
The Processor allows the user to configure an optional Message Delimiter that
|
||||||
can be used to send many messages per FlowFile. For example, a \n could be used
|
can be used to send many messages per FlowFile. For example, a \n could be used
|
||||||
to indicate that the contents of the FlowFile should be used to send one message
|
to indicate that the contents of the FlowFile should be used to send one message
|
||||||
per line of text. If the property is not set, the entire contents of the FlowFile
|
per line of text. If the property is not set, the entire contents of the FlowFile
|
||||||
will be sent as a single message. When using the delimiter, if some messages are
|
will be sent as a single message. When using the delimiter, if some messages are
|
||||||
successfully sent but other messages fail to send, the FlowFile will be FORKed into
|
successfully sent but other messages fail to send, the FlowFile will be FORKed into
|
||||||
two child FlowFiles, with the successfully sent messages being routed to 'success'
|
two child FlowFiles, with the successfully sent messages being routed to 'success'
|
||||||
and the messages that could not be sent going to 'failure'.
|
and the messages that could not be sent going to 'failure'.
|
||||||
</p>
|
</p>
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
|
|
@ -37,14 +37,13 @@ import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
public class TestGetKafka {
|
public class TestGetKafka {
|
||||||
|
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void configureLogging() {
|
public static void configureLogging() {
|
||||||
System.setProperty("org.slf4j.simpleLogger.log.kafka", "INFO");
|
System.setProperty("org.slf4j.simpleLogger.log.kafka", "INFO");
|
||||||
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.kafka", "INFO");
|
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.kafka", "INFO");
|
||||||
BasicConfigurator.configure();
|
BasicConfigurator.configure();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Ignore("Intended only for local tests to verify functionality.")
|
@Ignore("Intended only for local tests to verify functionality.")
|
||||||
public void testIntegrationLocally() {
|
public void testIntegrationLocally() {
|
||||||
|
@ -53,24 +52,23 @@ public class TestGetKafka {
|
||||||
runner.setProperty(GetKafka.TOPIC, "testX");
|
runner.setProperty(GetKafka.TOPIC, "testX");
|
||||||
runner.setProperty(GetKafka.KAFKA_TIMEOUT, "3 secs");
|
runner.setProperty(GetKafka.KAFKA_TIMEOUT, "3 secs");
|
||||||
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
||||||
|
|
||||||
runner.run(20, false);
|
runner.run(20, false);
|
||||||
|
|
||||||
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS);
|
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS);
|
||||||
for ( final MockFlowFile flowFile : flowFiles ) {
|
for (final MockFlowFile flowFile : flowFiles) {
|
||||||
System.out.println(flowFile.getAttributes());
|
System.out.println(flowFile.getAttributes());
|
||||||
System.out.println(new String(flowFile.toByteArray()));
|
System.out.println(new String(flowFile.toByteArray()));
|
||||||
System.out.println();
|
System.out.println();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithDelimiter() {
|
public void testWithDelimiter() {
|
||||||
final List<String> messages = new ArrayList<>();
|
final List<String> messages = new ArrayList<>();
|
||||||
messages.add("Hello");
|
messages.add("Hello");
|
||||||
messages.add("Good-bye");
|
messages.add("Good-bye");
|
||||||
|
|
||||||
final TestableProcessor proc = new TestableProcessor(null, messages);
|
final TestableProcessor proc = new TestableProcessor(null, messages);
|
||||||
final TestRunner runner = TestRunners.newTestRunner(proc);
|
final TestRunner runner = TestRunners.newTestRunner(proc);
|
||||||
runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "localhost:2181");
|
runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "localhost:2181");
|
||||||
|
@ -79,20 +77,20 @@ public class TestGetKafka {
|
||||||
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
||||||
runner.setProperty(GetKafka.MESSAGE_DEMARCATOR, "\\n");
|
runner.setProperty(GetKafka.MESSAGE_DEMARCATOR, "\\n");
|
||||||
runner.setProperty(GetKafka.BATCH_SIZE, "2");
|
runner.setProperty(GetKafka.BATCH_SIZE, "2");
|
||||||
|
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(GetKafka.REL_SUCCESS, 1);
|
runner.assertAllFlowFilesTransferred(GetKafka.REL_SUCCESS, 1);
|
||||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS).get(0);
|
final MockFlowFile mff = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS).get(0);
|
||||||
mff.assertContentEquals("Hello\nGood-bye");
|
mff.assertContentEquals("Hello\nGood-bye");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithDelimiterAndNotEnoughMessages() {
|
public void testWithDelimiterAndNotEnoughMessages() {
|
||||||
final List<String> messages = new ArrayList<>();
|
final List<String> messages = new ArrayList<>();
|
||||||
messages.add("Hello");
|
messages.add("Hello");
|
||||||
messages.add("Good-bye");
|
messages.add("Good-bye");
|
||||||
|
|
||||||
final TestableProcessor proc = new TestableProcessor(null, messages);
|
final TestableProcessor proc = new TestableProcessor(null, messages);
|
||||||
final TestRunner runner = TestRunners.newTestRunner(proc);
|
final TestRunner runner = TestRunners.newTestRunner(proc);
|
||||||
runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "localhost:2181");
|
runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "localhost:2181");
|
||||||
|
@ -101,40 +99,40 @@ public class TestGetKafka {
|
||||||
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
runner.setProperty(GetKafka.ZOOKEEPER_TIMEOUT, "3 secs");
|
||||||
runner.setProperty(GetKafka.MESSAGE_DEMARCATOR, "\\n");
|
runner.setProperty(GetKafka.MESSAGE_DEMARCATOR, "\\n");
|
||||||
runner.setProperty(GetKafka.BATCH_SIZE, "3");
|
runner.setProperty(GetKafka.BATCH_SIZE, "3");
|
||||||
|
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(GetKafka.REL_SUCCESS, 1);
|
runner.assertAllFlowFilesTransferred(GetKafka.REL_SUCCESS, 1);
|
||||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS).get(0);
|
final MockFlowFile mff = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS).get(0);
|
||||||
mff.assertContentEquals("Hello\nGood-bye");
|
mff.assertContentEquals("Hello\nGood-bye");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private static class TestableProcessor extends GetKafka {
|
private static class TestableProcessor extends GetKafka {
|
||||||
|
|
||||||
private final byte[] key;
|
private final byte[] key;
|
||||||
private final Iterator<String> messageItr;
|
private final Iterator<String> messageItr;
|
||||||
|
|
||||||
public TestableProcessor(final byte[] key, final List<String> messages) {
|
public TestableProcessor(final byte[] key, final List<String> messages) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
messageItr = messages.iterator();
|
messageItr = messages.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void createConsumers(ProcessContext context) {
|
public void createConsumers(ProcessContext context) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||||
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
|
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
|
||||||
final ConsumerIterator<byte[], byte[]> itr = Mockito.mock(ConsumerIterator.class);
|
final ConsumerIterator<byte[], byte[]> itr = Mockito.mock(ConsumerIterator.class);
|
||||||
|
|
||||||
Mockito.doAnswer(new Answer<Boolean>() {
|
Mockito.doAnswer(new Answer<Boolean>() {
|
||||||
@Override
|
@Override
|
||||||
public Boolean answer(final InvocationOnMock invocation) throws Throwable {
|
public Boolean answer(final InvocationOnMock invocation) throws Throwable {
|
||||||
return messageItr.hasNext();
|
return messageItr.hasNext();
|
||||||
}
|
}
|
||||||
}).when(itr).hasNext();
|
}).when(itr).hasNext();
|
||||||
|
|
||||||
Mockito.doAnswer(new Answer<MessageAndMetadata>() {
|
Mockito.doAnswer(new Answer<MessageAndMetadata>() {
|
||||||
@Override
|
@Override
|
||||||
public MessageAndMetadata answer(InvocationOnMock invocation) throws Throwable {
|
public MessageAndMetadata answer(InvocationOnMock invocation) throws Throwable {
|
||||||
|
@ -142,21 +140,21 @@ public class TestGetKafka {
|
||||||
Mockito.when(mam.key()).thenReturn(key);
|
Mockito.when(mam.key()).thenReturn(key);
|
||||||
Mockito.when(mam.offset()).thenReturn(0L);
|
Mockito.when(mam.offset()).thenReturn(0L);
|
||||||
Mockito.when(mam.partition()).thenReturn(0);
|
Mockito.when(mam.partition()).thenReturn(0);
|
||||||
|
|
||||||
Mockito.doAnswer(new Answer<byte[]>() {
|
Mockito.doAnswer(new Answer<byte[]>() {
|
||||||
@Override
|
@Override
|
||||||
public byte[] answer(InvocationOnMock invocation) throws Throwable {
|
public byte[] answer(InvocationOnMock invocation) throws Throwable {
|
||||||
return messageItr.next().getBytes();
|
return messageItr.next().getBytes();
|
||||||
}
|
}
|
||||||
|
|
||||||
}).when(mam).message();
|
}).when(mam).message();
|
||||||
|
|
||||||
return mam;
|
return mam;
|
||||||
}
|
}
|
||||||
}).when(itr).next();
|
}).when(itr).next();
|
||||||
|
|
||||||
return itr;
|
return itr;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,13 +36,19 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||||
|
|
||||||
import org.apache.nifi.processor.ProcessContext;
|
import org.apache.nifi.processor.ProcessContext;
|
||||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||||
import org.apache.nifi.util.*;
|
import org.apache.nifi.util.MockFlowFile;
|
||||||
|
import org.apache.nifi.util.MockFlowFileQueue;
|
||||||
|
import org.apache.nifi.util.MockProcessSession;
|
||||||
|
import org.apache.nifi.util.MockProvenanceReporter;
|
||||||
|
import org.apache.nifi.util.MockSessionFactory;
|
||||||
|
import org.apache.nifi.util.SharedSessionState;
|
||||||
|
import org.apache.nifi.util.TestRunner;
|
||||||
|
import org.apache.nifi.util.TestRunners;
|
||||||
import org.junit.Ignore;
|
import org.junit.Ignore;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
import org.mockito.internal.util.reflection.Whitebox;
|
import org.mockito.internal.util.reflection.Whitebox;
|
||||||
|
|
||||||
|
|
||||||
public class TestPutKafka {
|
public class TestPutKafka {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -53,15 +59,15 @@ public class TestPutKafka {
|
||||||
runner.setProperty(PutKafka.KEY, "key1");
|
runner.setProperty(PutKafka.KEY, "key1");
|
||||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
||||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
||||||
|
|
||||||
runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5\n6\n7\n8\n9".getBytes());
|
runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5\n6\n7\n8\n9".getBytes());
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
||||||
|
|
||||||
final List<byte[]> messages = proc.getProducer().getMessages();
|
final List<byte[]> messages = proc.getProducer().getMessages();
|
||||||
assertEquals(11, messages.size());
|
assertEquals(11, messages.size());
|
||||||
|
|
||||||
assertTrue(Arrays.equals("Hello World".getBytes(StandardCharsets.UTF_8), messages.get(0)));
|
assertTrue(Arrays.equals("Hello World".getBytes(StandardCharsets.UTF_8), messages.get(0)));
|
||||||
assertTrue(Arrays.equals("Goodbye".getBytes(StandardCharsets.UTF_8), messages.get(1)));
|
assertTrue(Arrays.equals("Goodbye".getBytes(StandardCharsets.UTF_8), messages.get(1)));
|
||||||
assertTrue(Arrays.equals("1".getBytes(StandardCharsets.UTF_8), messages.get(2)));
|
assertTrue(Arrays.equals("1".getBytes(StandardCharsets.UTF_8), messages.get(2)));
|
||||||
|
@ -74,8 +80,7 @@ public class TestPutKafka {
|
||||||
assertTrue(Arrays.equals("8".getBytes(StandardCharsets.UTF_8), messages.get(9)));
|
assertTrue(Arrays.equals("8".getBytes(StandardCharsets.UTF_8), messages.get(9)));
|
||||||
assertTrue(Arrays.equals("9".getBytes(StandardCharsets.UTF_8), messages.get(10)));
|
assertTrue(Arrays.equals("9".getBytes(StandardCharsets.UTF_8), messages.get(10)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithImmediateFailure() {
|
public void testWithImmediateFailure() {
|
||||||
final TestableProcessor proc = new TestableProcessor(0);
|
final TestableProcessor proc = new TestableProcessor(0);
|
||||||
|
@ -84,17 +89,16 @@ public class TestPutKafka {
|
||||||
runner.setProperty(PutKafka.KEY, "key1");
|
runner.setProperty(PutKafka.KEY, "key1");
|
||||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
||||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
||||||
|
|
||||||
final String text = "Hello World\nGoodbye\n1\n2\n3\n4\n5\n6\n7\n8\n9";
|
final String text = "Hello World\nGoodbye\n1\n2\n3\n4\n5\n6\n7\n8\n9";
|
||||||
runner.enqueue(text.getBytes());
|
runner.enqueue(text.getBytes());
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_FAILURE, 1);
|
runner.assertAllFlowFilesTransferred(PutKafka.REL_FAILURE, 1);
|
||||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(PutKafka.REL_FAILURE).get(0);
|
final MockFlowFile mff = runner.getFlowFilesForRelationship(PutKafka.REL_FAILURE).get(0);
|
||||||
mff.assertContentEquals(text);
|
mff.assertContentEquals(text);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPartialFailure() {
|
public void testPartialFailure() {
|
||||||
final TestableProcessor proc = new TestableProcessor(2);
|
final TestableProcessor proc = new TestableProcessor(2);
|
||||||
|
@ -104,22 +108,21 @@ public class TestPutKafka {
|
||||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
||||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
||||||
runner.setProperty(PutKafka.MAX_BUFFER_SIZE, "1 B");
|
runner.setProperty(PutKafka.MAX_BUFFER_SIZE, "1 B");
|
||||||
|
|
||||||
final byte[] bytes = "1\n2\n3\n4".getBytes();
|
final byte[] bytes = "1\n2\n3\n4".getBytes();
|
||||||
runner.enqueue(bytes);
|
runner.enqueue(bytes);
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertTransferCount(PutKafka.REL_SUCCESS, 1);
|
runner.assertTransferCount(PutKafka.REL_SUCCESS, 1);
|
||||||
runner.assertTransferCount(PutKafka.REL_FAILURE, 1);
|
runner.assertTransferCount(PutKafka.REL_FAILURE, 1);
|
||||||
|
|
||||||
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0);
|
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0);
|
||||||
successFF.assertContentEquals("1\n2\n");
|
successFF.assertContentEquals("1\n2\n");
|
||||||
|
|
||||||
final MockFlowFile failureFF = runner.getFlowFilesForRelationship(PutKafka.REL_FAILURE).get(0);
|
final MockFlowFile failureFF = runner.getFlowFilesForRelationship(PutKafka.REL_FAILURE).get(0);
|
||||||
failureFF.assertContentEquals("3\n4");
|
failureFF.assertContentEquals("3\n4");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithEmptyMessages() {
|
public void testWithEmptyMessages() {
|
||||||
final TestableProcessor proc = new TestableProcessor();
|
final TestableProcessor proc = new TestableProcessor();
|
||||||
|
@ -128,11 +131,11 @@ public class TestPutKafka {
|
||||||
runner.setProperty(PutKafka.KEY, "key1");
|
runner.setProperty(PutKafka.KEY, "key1");
|
||||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
||||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n");
|
||||||
|
|
||||||
final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes();
|
final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes();
|
||||||
runner.enqueue(bytes);
|
runner.enqueue(bytes);
|
||||||
runner.run();
|
runner.run();
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
||||||
|
|
||||||
final List<byte[]> msgs = proc.getProducer().getMessages();
|
final List<byte[]> msgs = proc.getProducer().getMessages();
|
||||||
|
@ -144,7 +147,7 @@ public class TestPutKafka {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testProvenanceReporterMessagesCount(){
|
public void testProvenanceReporterMessagesCount() {
|
||||||
final TestableProcessor processor = new TestableProcessor();
|
final TestableProcessor processor = new TestableProcessor();
|
||||||
|
|
||||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||||
|
@ -157,7 +160,6 @@ public class TestPutKafka {
|
||||||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||||
|
|
||||||
|
|
||||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||||
|
@ -176,7 +178,7 @@ public class TestPutKafka {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testProvenanceReporterWithoutDelimiterMessagesCount(){
|
public void testProvenanceReporterWithoutDelimiterMessagesCount() {
|
||||||
final TestableProcessor processor = new TestableProcessor();
|
final TestableProcessor processor = new TestableProcessor();
|
||||||
|
|
||||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||||
|
@ -189,7 +191,6 @@ public class TestPutKafka {
|
||||||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||||
|
|
||||||
|
|
||||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||||
|
@ -206,97 +207,97 @@ public class TestPutKafka {
|
||||||
Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1");
|
Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...")
|
@Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...")
|
||||||
public void testKeyValuePut() {
|
public void testKeyValuePut() {
|
||||||
final TestRunner runner = TestRunners.newTestRunner(PutKafka.class);
|
final TestRunner runner = TestRunners.newTestRunner(PutKafka.class);
|
||||||
runner.setProperty(PutKafka.SEED_BROKERS, "192.168.0.101:9092");
|
runner.setProperty(PutKafka.SEED_BROKERS, "192.168.0.101:9092");
|
||||||
runner.setProperty(PutKafka.TOPIC, "${kafka.topic}");
|
runner.setProperty(PutKafka.TOPIC, "${kafka.topic}");
|
||||||
runner.setProperty(PutKafka.KEY, "${kafka.key}");
|
runner.setProperty(PutKafka.KEY, "${kafka.key}");
|
||||||
runner.setProperty(PutKafka.TIMEOUT, "3 secs");
|
runner.setProperty(PutKafka.TIMEOUT, "3 secs");
|
||||||
runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue());
|
runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue());
|
||||||
|
|
||||||
final Map<String, String> attributes = new HashMap<>();
|
final Map<String, String> attributes = new HashMap<>();
|
||||||
attributes.put("kafka.topic", "test");
|
attributes.put("kafka.topic", "test");
|
||||||
attributes.put("kafka.key", "key3");
|
attributes.put("kafka.key", "key3");
|
||||||
|
|
||||||
final byte[] data = "Hello, World, Again! ;)".getBytes();
|
final byte[] data = "Hello, World, Again! ;)".getBytes();
|
||||||
runner.enqueue(data, attributes);
|
runner.enqueue(data, attributes);
|
||||||
runner.enqueue(data, attributes);
|
runner.enqueue(data, attributes);
|
||||||
runner.enqueue(data, attributes);
|
runner.enqueue(data, attributes);
|
||||||
runner.enqueue(data, attributes);
|
runner.enqueue(data, attributes);
|
||||||
|
|
||||||
runner.run(5);
|
runner.run(5);
|
||||||
|
|
||||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 4);
|
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 4);
|
||||||
final List<MockFlowFile> mffs = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS);
|
final List<MockFlowFile> mffs = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS);
|
||||||
final MockFlowFile mff = mffs.get(0);
|
final MockFlowFile mff = mffs.get(0);
|
||||||
|
|
||||||
assertTrue(Arrays.equals(data, mff.toByteArray()));
|
assertTrue(Arrays.equals(data, mff.toByteArray()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class TestableProcessor extends PutKafka {
|
||||||
private static class TestableProcessor extends PutKafka {
|
|
||||||
private MockProducer producer;
|
private MockProducer producer;
|
||||||
private int failAfter = Integer.MAX_VALUE;
|
private int failAfter = Integer.MAX_VALUE;
|
||||||
|
|
||||||
public TestableProcessor() {
|
public TestableProcessor() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public TestableProcessor(final int failAfter) {
|
public TestableProcessor(final int failAfter) {
|
||||||
this.failAfter = failAfter;
|
this.failAfter = failAfter;
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnScheduled
|
@OnScheduled
|
||||||
public void instantiateProducer(final ProcessContext context) {
|
public void instantiateProducer(final ProcessContext context) {
|
||||||
producer = new MockProducer(createConfig(context));
|
producer = new MockProducer(createConfig(context));
|
||||||
producer.setFailAfter(failAfter);
|
producer.setFailAfter(failAfter);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
||||||
return producer;
|
return producer;
|
||||||
}
|
}
|
||||||
|
|
||||||
public MockProducer getProducer() {
|
public MockProducer getProducer() {
|
||||||
return producer;
|
return producer;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class MockProducer extends Producer<byte[], byte[]> {
|
||||||
private static class MockProducer extends Producer<byte[], byte[]> {
|
|
||||||
private int sendCount = 0;
|
private int sendCount = 0;
|
||||||
private int failAfter = Integer.MAX_VALUE;
|
private int failAfter = Integer.MAX_VALUE;
|
||||||
|
|
||||||
private final List<byte[]> messages = new ArrayList<>();
|
private final List<byte[]> messages = new ArrayList<>();
|
||||||
|
|
||||||
public MockProducer(final ProducerConfig config) {
|
public MockProducer(final ProducerConfig config) {
|
||||||
super(config);
|
super(config);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void send(final KeyedMessage<byte[], byte[]> message) {
|
public void send(final KeyedMessage<byte[], byte[]> message) {
|
||||||
if ( ++sendCount > failAfter ) {
|
if (++sendCount > failAfter) {
|
||||||
throw new FailedToSendMessageException("Failed to send message", new RuntimeException("Unit test told to fail after " + failAfter + " successful messages"));
|
throw new FailedToSendMessageException("Failed to send message", new RuntimeException("Unit test told to fail after " + failAfter + " successful messages"));
|
||||||
} else {
|
} else {
|
||||||
messages.add(message.message());
|
messages.add(message.message());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<byte[]> getMessages() {
|
public List<byte[]> getMessages() {
|
||||||
return messages;
|
return messages;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void send(final List<KeyedMessage<byte[], byte[]>> messages) {
|
public void send(final List<KeyedMessage<byte[], byte[]>> messages) {
|
||||||
for ( final KeyedMessage<byte[], byte[]> msg : messages ) {
|
for (final KeyedMessage<byte[], byte[]> msg : messages) {
|
||||||
send(msg);
|
send(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFailAfter(final int successCount) {
|
public void setFailAfter(final int successCount) {
|
||||||
failAfter = successCount;
|
failAfter = successCount;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,12 +26,12 @@
|
||||||
<module>nifi-kafka-nar</module>
|
<module>nifi-kafka-nar</module>
|
||||||
</modules>
|
</modules>
|
||||||
<dependencyManagement>
|
<dependencyManagement>
|
||||||
<dependencies>
|
<dependencies>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-kafka-processors</artifactId>
|
<artifactId>nifi-kafka-processors</artifactId>
|
||||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
</dependencyManagement>
|
</dependencyManagement>
|
||||||
</project>
|
</project>
|
||||||
|
|
Loading…
Reference in New Issue