mirror of https://github.com/apache/nifi.git
NIFI-271
This commit is contained in:
parent
b2a1f5217d
commit
8f2502c4e4
|
@ -61,82 +61,86 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@SupportsBatching
|
||||
@CapabilityDescription("Fetches messages from Apache Kafka")
|
||||
@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"),
|
||||
@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.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") })
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"),
|
||||
@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.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 static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
|
||||
.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")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
.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")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
public static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
||||
.name("Topic Name")
|
||||
.description("The Kafka Topic to pull messages from")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
.name("Topic Name")
|
||||
.description("The Kafka Topic to pull messages from")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
public static final PropertyDescriptor ZOOKEEPER_COMMIT_DELAY = new PropertyDescriptor.Builder()
|
||||
.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")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("60 secs")
|
||||
.build();
|
||||
.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")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("60 secs")
|
||||
.build();
|
||||
public static final PropertyDescriptor ZOOKEEPER_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("ZooKeeper Communications Timeout")
|
||||
.description("The amount of time to wait for a response from ZooKeeper before determining that there is a communications error")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("30 secs")
|
||||
.build();
|
||||
.name("ZooKeeper Communications Timeout")
|
||||
.description("The amount of time to wait for a response from ZooKeeper 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 KAFKA_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("Kafka 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();
|
||||
.name("Kafka 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 BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("Batch Size")
|
||||
.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. "
|
||||
+ "If the messages from Kafka should not be concatenated together, leave this value at 1.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("1")
|
||||
.build();
|
||||
.name("Batch Size")
|
||||
.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. "
|
||||
+ "If the messages from Kafka should not be concatenated together, leave this value at 1.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("1")
|
||||
.build();
|
||||
public static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor.Builder()
|
||||
.name("Message Demarcator")
|
||||
.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, "
|
||||
+ "this value will be placed in between them.")
|
||||
.required(true)
|
||||
.addValidator(Validator.VALID) // accept anything as a demarcator, including empty string
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("\\n")
|
||||
.build();
|
||||
.name("Message Demarcator")
|
||||
.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, "
|
||||
+ "this value will be placed in between them.")
|
||||
.required(true)
|
||||
.addValidator(Validator.VALID) // accept anything as a demarcator, including empty string
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("\\n")
|
||||
.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();
|
||||
.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()
|
||||
.name("success")
|
||||
.description("All FlowFiles that are created are routed to this relationship")
|
||||
.build();
|
||||
|
||||
.name("success")
|
||||
.description("All FlowFiles that are created are routed to this relationship")
|
||||
.build();
|
||||
|
||||
private final BlockingQueue<ConsumerIterator<byte[], byte[]>> streamIterators = new LinkedBlockingQueue<>();
|
||||
private volatile ConsumerConnector consumer;
|
||||
|
@ -147,10 +151,10 @@ public class GetKafka extends AbstractProcessor {
|
|||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder()
|
||||
.fromPropertyDescriptor(CLIENT_NAME)
|
||||
.defaultValue("NiFi-" + getIdentifier())
|
||||
.build();
|
||||
final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder()
|
||||
.fromPropertyDescriptor(CLIENT_NAME)
|
||||
.defaultValue("NiFi-" + getIdentifier())
|
||||
.build();
|
||||
|
||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(ZOOKEEPER_CONNECTION_STRING);
|
||||
|
@ -173,57 +177,57 @@ public class GetKafka extends AbstractProcessor {
|
|||
|
||||
@OnScheduled
|
||||
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);
|
||||
topicCountMap.put(topic, context.getMaxConcurrentTasks());
|
||||
final Map<String, Integer> topicCountMap = new HashMap<>(1);
|
||||
topicCountMap.put(topic, context.getMaxConcurrentTasks());
|
||||
|
||||
final Properties props = new Properties();
|
||||
props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue());
|
||||
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.enable", "true"); // just be explicit
|
||||
props.setProperty("auto.offset.reset", "smallest");
|
||||
final Properties props = new Properties();
|
||||
props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue());
|
||||
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.enable", "true"); // just be explicit
|
||||
props.setProperty("auto.offset.reset", "smallest");
|
||||
|
||||
final ConsumerConfig consumerConfig = new ConsumerConfig(props);
|
||||
consumer = Consumer.createJavaConsumerConnector(consumerConfig);
|
||||
final ConsumerConfig consumerConfig = new ConsumerConfig(props);
|
||||
consumer = Consumer.createJavaConsumerConnector(consumerConfig);
|
||||
|
||||
final Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
|
||||
final List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
|
||||
final Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
|
||||
final List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
|
||||
|
||||
this.streamIterators.clear();
|
||||
this.streamIterators.clear();
|
||||
|
||||
for ( final KafkaStream<byte[], byte[]> stream : streams ) {
|
||||
streamIterators.add(stream.iterator());
|
||||
}
|
||||
for (final KafkaStream<byte[], byte[]> stream : streams) {
|
||||
streamIterators.add(stream.iterator());
|
||||
}
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void shutdownConsumer() {
|
||||
if ( consumer != null ) {
|
||||
try {
|
||||
consumer.commitOffsets();
|
||||
} finally {
|
||||
consumer.shutdown();
|
||||
}
|
||||
}
|
||||
if (consumer != null) {
|
||||
try {
|
||||
consumer.commitOffsets();
|
||||
} finally {
|
||||
consumer.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@OnUnscheduled
|
||||
public void interruptIterators() {
|
||||
// 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
|
||||
// ability to shutdown the Processor.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
for ( final Thread t : interruptableThreads ) {
|
||||
t.interrupt();
|
||||
}
|
||||
// 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
|
||||
// ability to shutdown the Processor.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
for (final Thread t : interruptableThreads) {
|
||||
t.interrupt();
|
||||
}
|
||||
|
||||
interruptableThreads.clear();
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
interruptableThreads.clear();
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
|
||||
|
@ -232,105 +236,105 @@ public class GetKafka extends AbstractProcessor {
|
|||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
ConsumerIterator<byte[], byte[]> iterator = getStreamIterator();
|
||||
if ( iterator == null ) {
|
||||
return;
|
||||
}
|
||||
ConsumerIterator<byte[], byte[]> iterator = getStreamIterator();
|
||||
if (iterator == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
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 byte[] demarcatorBytes = demarcator.getBytes(StandardCharsets.UTF_8);
|
||||
final String topic = context.getProperty(TOPIC).getValue();
|
||||
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 byte[] demarcatorBytes = demarcator.getBytes(StandardCharsets.UTF_8);
|
||||
final String topic = context.getProperty(TOPIC).getValue();
|
||||
|
||||
FlowFile flowFile = null;
|
||||
try {
|
||||
// add the current thread to the Set of those to be interrupted if processor stopped.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
interruptableThreads.add(Thread.currentThread());
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
FlowFile flowFile = null;
|
||||
try {
|
||||
// add the current thread to the Set of those to be interrupted if processor stopped.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
interruptableThreads.add(Thread.currentThread());
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
|
||||
final long start = System.nanoTime();
|
||||
flowFile = session.create();
|
||||
final long start = System.nanoTime();
|
||||
flowFile = session.create();
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("kafka.topic", topic);
|
||||
|
||||
int numMessages = 0;
|
||||
for (int msgCount = 0; msgCount < batchSize; msgCount++) {
|
||||
// if the processor is stopped, iterator.hasNext() will throw an Exception.
|
||||
// In this case, we just break out of the loop.
|
||||
try {
|
||||
if ( !iterator.hasNext() ) {
|
||||
break;
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
break;
|
||||
}
|
||||
for (int msgCount = 0; msgCount < batchSize; msgCount++) {
|
||||
// if the processor is stopped, iterator.hasNext() will throw an Exception.
|
||||
// In this case, we just break out of the loop.
|
||||
try {
|
||||
if (!iterator.hasNext()) {
|
||||
break;
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
break;
|
||||
}
|
||||
|
||||
final MessageAndMetadata<byte[], byte[]> mam = iterator.next();
|
||||
if ( mam == null ) {
|
||||
return;
|
||||
}
|
||||
final MessageAndMetadata<byte[], byte[]> mam = iterator.next();
|
||||
if (mam == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final byte[] key = mam.key();
|
||||
final byte[] key = mam.key();
|
||||
|
||||
if ( batchSize == 1 ) {
|
||||
// the kafka.key, kafka.offset, and kafka.partition attributes are added only
|
||||
// for a batch size of 1.
|
||||
if ( key != null ) {
|
||||
attributes.put("kafka.key", new String(key, StandardCharsets.UTF_8));
|
||||
}
|
||||
if (batchSize == 1) {
|
||||
// the kafka.key, kafka.offset, and kafka.partition attributes are added only
|
||||
// for a batch size of 1.
|
||||
if (key != null) {
|
||||
attributes.put("kafka.key", new String(key, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
attributes.put("kafka.offset", String.valueOf(mam.offset()));
|
||||
attributes.put("kafka.partition", String.valueOf(mam.partition()));
|
||||
}
|
||||
attributes.put("kafka.offset", String.valueOf(mam.offset()));
|
||||
attributes.put("kafka.partition", String.valueOf(mam.partition()));
|
||||
}
|
||||
|
||||
// add the message to the FlowFile's contents
|
||||
final boolean firstMessage = (msgCount == 0);
|
||||
flowFile = session.append(flowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
if ( !firstMessage ) {
|
||||
out.write(demarcatorBytes);
|
||||
}
|
||||
out.write(mam.message());
|
||||
}
|
||||
});
|
||||
numMessages++;
|
||||
}
|
||||
// add the message to the FlowFile's contents
|
||||
final boolean firstMessage = (msgCount == 0);
|
||||
flowFile = session.append(flowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
if (!firstMessage) {
|
||||
out.write(demarcatorBytes);
|
||||
}
|
||||
out.write(mam.message());
|
||||
}
|
||||
});
|
||||
numMessages++;
|
||||
}
|
||||
|
||||
// If we received no messages, remove the FlowFile. Otherwise, send to success.
|
||||
if ( flowFile.getSize() == 0L ) {
|
||||
session.remove(flowFile);
|
||||
} else {
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
||||
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});
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
getLogger().error("Failed to receive FlowFile from Kafka due to {}", new Object[] {e});
|
||||
if ( flowFile != null ) {
|
||||
session.remove(flowFile);
|
||||
}
|
||||
} finally {
|
||||
// Remove the current thread from the Set of Threads to interrupt.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
interruptableThreads.remove(Thread.currentThread());
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
// If we received no messages, remove the FlowFile. Otherwise, send to success.
|
||||
if (flowFile.getSize() == 0L) {
|
||||
session.remove(flowFile);
|
||||
} else {
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
||||
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});
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
getLogger().error("Failed to receive FlowFile from Kafka due to {}", new Object[]{e});
|
||||
if (flowFile != null) {
|
||||
session.remove(flowFile);
|
||||
}
|
||||
} finally {
|
||||
// Remove the current thread from the Set of Threads to interrupt.
|
||||
interruptionLock.lock();
|
||||
try {
|
||||
interruptableThreads.remove(Thread.currentThread());
|
||||
} finally {
|
||||
interruptionLock.unlock();
|
||||
}
|
||||
|
||||
// Add the iterator back to the queue
|
||||
if ( iterator != null ) {
|
||||
streamIterators.offer(iterator);
|
||||
}
|
||||
}
|
||||
// Add the iterator back to the queue
|
||||
if (iterator != null) {
|
||||
streamIterators.offer(iterator);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -61,94 +61,99 @@ import scala.actors.threadpool.Arrays;
|
|||
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub"})
|
||||
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka")
|
||||
public class PutKafka extends AbstractProcessor {
|
||||
|
||||
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
|
||||
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 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();
|
||||
.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();
|
||||
.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();
|
||||
.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();
|
||||
.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();
|
||||
.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();
|
||||
.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();
|
||||
.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();
|
||||
|
||||
.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()
|
||||
.name("success")
|
||||
.description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
.name("success")
|
||||
.description("Any FlowFile that is successfully sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
.name("failure")
|
||||
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
|
||||
private final BlockingQueue<Producer<byte[], byte[]>> producers = new LinkedBlockingQueue<>();
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final PropertyDescriptor clientName = new PropertyDescriptor.Builder()
|
||||
.fromPropertyDescriptor(CLIENT_NAME)
|
||||
.defaultValue("NiFi-" + getIdentifier())
|
||||
.build();
|
||||
final PropertyDescriptor clientName = new PropertyDescriptor.Builder()
|
||||
.fromPropertyDescriptor(CLIENT_NAME)
|
||||
.defaultValue("NiFi-" + getIdentifier())
|
||||
.build();
|
||||
|
||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(SEED_BROKERS);
|
||||
|
@ -170,14 +175,13 @@ public class PutKafka extends AbstractProcessor {
|
|||
return relationships;
|
||||
}
|
||||
|
||||
|
||||
@OnStopped
|
||||
public void closeProducers() {
|
||||
Producer<byte[], byte[]> producer;
|
||||
Producer<byte[], byte[]> producer;
|
||||
|
||||
while ((producer = producers.poll()) != null) {
|
||||
producer.close();
|
||||
}
|
||||
while ((producer = producers.poll()) != null) {
|
||||
producer.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected ProducerConfig createConfig(final ProcessContext context) {
|
||||
|
@ -196,51 +200,51 @@ public class PutKafka extends AbstractProcessor {
|
|||
}
|
||||
|
||||
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) {
|
||||
Producer<byte[], byte[]> producer = producers.poll();
|
||||
return producer == null ? createProducer(context) : producer;
|
||||
Producer<byte[], byte[]> producer = producers.poll();
|
||||
return producer == null ? createProducer(context) : producer;
|
||||
}
|
||||
|
||||
private void returnProducer(final Producer<byte[], byte[]> producer) {
|
||||
producers.offer(producer);
|
||||
producers.offer(producer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
FlowFile flowFile = session.get();
|
||||
if ( flowFile == null ) {
|
||||
return;
|
||||
}
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final long start = System.nanoTime();
|
||||
final long start = System.nanoTime();
|
||||
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final byte[] keyBytes = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8);
|
||||
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");
|
||||
}
|
||||
|
||||
final long maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue();
|
||||
final Producer<byte[], byte[]> producer = borrowProducer(context);
|
||||
|
||||
if ( delimiter == null ) {
|
||||
if (delimiter == null) {
|
||||
// Send the entire FlowFile as a single message.
|
||||
final byte[] value = new byte[(int) flowFile.getSize()];
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
public void process(final InputStream in) throws IOException {
|
||||
StreamUtils.fillBuffer(in, value);
|
||||
}
|
||||
@Override
|
||||
public void process(final InputStream in) throws IOException {
|
||||
StreamUtils.fillBuffer(in, value);
|
||||
}
|
||||
});
|
||||
|
||||
boolean error = false;
|
||||
try {
|
||||
final KeyedMessage<byte[], byte[]> message;
|
||||
if ( key == null ) {
|
||||
if (key == null) {
|
||||
message = new KeyedMessage<>(topic, value);
|
||||
} else {
|
||||
message = new KeyedMessage<>(topic, keyBytes, value);
|
||||
|
@ -251,13 +255,13 @@ public class PutKafka extends AbstractProcessor {
|
|||
|
||||
session.getProvenanceReporter().send(flowFile, "kafka://" + topic);
|
||||
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) {
|
||||
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);
|
||||
error = true;
|
||||
} finally {
|
||||
if ( error ) {
|
||||
if (error) {
|
||||
producer.close();
|
||||
} else {
|
||||
returnProducer(producer);
|
||||
|
@ -288,13 +292,13 @@ public class PutKafka extends AbstractProcessor {
|
|||
|
||||
int nextByte;
|
||||
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.
|
||||
while (!streamFinished) {
|
||||
nextByte = in.read();
|
||||
|
||||
if ( nextByte > -1 ) {
|
||||
if (nextByte > -1) {
|
||||
baos.write(nextByte);
|
||||
}
|
||||
|
||||
|
@ -302,16 +306,16 @@ public class PutKafka extends AbstractProcessor {
|
|||
// we ran out of data. This message is complete.
|
||||
data = baos.toByteArray();
|
||||
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
|
||||
// underlying BAOS exception for the last 'delimiterBytes.length' bytes because we don't want
|
||||
// the delimiter itself to be sent.
|
||||
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 ( data.length != 0 ) {
|
||||
if (data.length != 0) {
|
||||
// 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.
|
||||
final KeyedMessage<byte[], byte[]> message;
|
||||
|
@ -361,7 +365,7 @@ public class PutKafka extends AbstractProcessor {
|
|||
}
|
||||
|
||||
// If there are messages left, send them
|
||||
if ( !messages.isEmpty() ) {
|
||||
if (!messages.isEmpty()) {
|
||||
try {
|
||||
messagesSent.addAndGet(messages.size()); // add count of messages
|
||||
producer.send(messages);
|
||||
|
@ -376,7 +380,7 @@ public class PutKafka extends AbstractProcessor {
|
|||
final long nanos = System.nanoTime() - start;
|
||||
session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages");
|
||||
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) {
|
||||
error = true;
|
||||
|
||||
|
@ -384,17 +388,18 @@ public class PutKafka extends AbstractProcessor {
|
|||
// 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'.
|
||||
final long offset = lastMessageOffset.get();
|
||||
if ( offset == 0L ) {
|
||||
if (offset == 0L) {
|
||||
// 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);
|
||||
} else {
|
||||
// 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 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[] {
|
||||
messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause() });
|
||||
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[]{
|
||||
messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause()});
|
||||
|
||||
session.transfer(successfulMessages, REL_SUCCESS);
|
||||
session.transfer(failedMessages, REL_FAILURE);
|
||||
|
@ -402,7 +407,7 @@ public class PutKafka extends AbstractProcessor {
|
|||
session.getProvenanceReporter().send(successfulMessages, "kafka://" + topic);
|
||||
}
|
||||
} finally {
|
||||
if ( error ) {
|
||||
if (error) {
|
||||
producer.close();
|
||||
} else {
|
||||
returnProducer(producer);
|
||||
|
|
|
@ -24,22 +24,22 @@
|
|||
<!-- Processor Documentation ================================================== -->
|
||||
<h2>Description:</h2>
|
||||
<p>
|
||||
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
|
||||
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>
|
||||
will be added to the FlowFile, with the value being the UTF-8 Encoded value
|
||||
of the Message's Key.
|
||||
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
|
||||
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>
|
||||
will be added to the FlowFile, with the value being the UTF-8 Encoded value
|
||||
of the Message's Key.
|
||||
</p>
|
||||
<p>
|
||||
Kafka supports the notion of a Consumer Group when pulling messages in order to
|
||||
provide scalability while still offering a publish-subscribe interface. Each
|
||||
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
|
||||
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
|
||||
from multiple Topics, as each Processor will receive a different Processor UUID
|
||||
and therefore a different Consumer Group Identifier.
|
||||
Kafka supports the notion of a Consumer Group when pulling messages in order to
|
||||
provide scalability while still offering a publish-subscribe interface. Each
|
||||
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
|
||||
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
|
||||
from multiple Topics, as each Processor will receive a different Processor UUID
|
||||
and therefore a different Consumer Group Identifier.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
|
|
|
@ -24,22 +24,22 @@
|
|||
<!-- Processor Documentation ================================================== -->
|
||||
<h2>Description:</h2>
|
||||
<p>
|
||||
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 FlowFile becomes the contents of a single message in Kafka.
|
||||
This message is optionally assigned a key by using the
|
||||
<Kafka Key> Property.
|
||||
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 FlowFile becomes the contents of a single message in Kafka.
|
||||
This message is optionally assigned a key by using the
|
||||
<Kafka Key> Property.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
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
|
||||
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
|
||||
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
|
||||
two child FlowFiles, with the successfully sent messages being routed to 'success'
|
||||
and the messages that could not be sent going to 'failure'.
|
||||
</p>
|
||||
<p>
|
||||
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
|
||||
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
|
||||
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
|
||||
two child FlowFiles, with the successfully sent messages being routed to 'success'
|
||||
and the messages that could not be sent going to 'failure'.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
||||
|
|
|
@ -37,10 +37,9 @@ import org.mockito.stubbing.Answer;
|
|||
|
||||
public class TestGetKafka {
|
||||
|
||||
|
||||
@BeforeClass
|
||||
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");
|
||||
BasicConfigurator.configure();
|
||||
}
|
||||
|
@ -57,14 +56,13 @@ public class TestGetKafka {
|
|||
runner.run(20, false);
|
||||
|
||||
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(GetKafka.REL_SUCCESS);
|
||||
for ( final MockFlowFile flowFile : flowFiles ) {
|
||||
System.out.println(flowFile.getAttributes());
|
||||
System.out.println(new String(flowFile.toByteArray()));
|
||||
System.out.println();
|
||||
for (final MockFlowFile flowFile : flowFiles) {
|
||||
System.out.println(flowFile.getAttributes());
|
||||
System.out.println(new String(flowFile.toByteArray()));
|
||||
System.out.println();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWithDelimiter() {
|
||||
final List<String> messages = new ArrayList<>();
|
||||
|
@ -109,8 +107,8 @@ public class TestGetKafka {
|
|||
mff.assertContentEquals("Hello\nGood-bye");
|
||||
}
|
||||
|
||||
|
||||
private static class TestableProcessor extends GetKafka {
|
||||
|
||||
private final byte[] key;
|
||||
private final Iterator<String> messageItr;
|
||||
|
||||
|
|
|
@ -36,13 +36,19 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
|||
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
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.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
|
||||
|
||||
public class TestPutKafka {
|
||||
|
||||
@Test
|
||||
|
@ -75,7 +81,6 @@ public class TestPutKafka {
|
|||
assertTrue(Arrays.equals("9".getBytes(StandardCharsets.UTF_8), messages.get(10)));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWithImmediateFailure() {
|
||||
final TestableProcessor proc = new TestableProcessor(0);
|
||||
|
@ -94,7 +99,6 @@ public class TestPutKafka {
|
|||
mff.assertContentEquals(text);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testPartialFailure() {
|
||||
final TestableProcessor proc = new TestableProcessor(2);
|
||||
|
@ -119,7 +123,6 @@ public class TestPutKafka {
|
|||
failureFF.assertContentEquals("3\n4");
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testWithEmptyMessages() {
|
||||
final TestableProcessor proc = new TestableProcessor();
|
||||
|
@ -144,7 +147,7 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testProvenanceReporterMessagesCount(){
|
||||
public void testProvenanceReporterMessagesCount() {
|
||||
final TestableProcessor processor = new TestableProcessor();
|
||||
|
||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||
|
@ -157,7 +160,6 @@ public class TestPutKafka {
|
|||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||
|
@ -176,7 +178,7 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testProvenanceReporterWithoutDelimiterMessagesCount(){
|
||||
public void testProvenanceReporterWithoutDelimiterMessagesCount() {
|
||||
final TestableProcessor processor = new TestableProcessor();
|
||||
|
||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||
|
@ -189,7 +191,6 @@ public class TestPutKafka {
|
|||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||
|
@ -206,69 +207,69 @@ public class TestPutKafka {
|
|||
Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1");
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...")
|
||||
public void testKeyValuePut() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutKafka.class);
|
||||
runner.setProperty(PutKafka.SEED_BROKERS, "192.168.0.101:9092");
|
||||
runner.setProperty(PutKafka.TOPIC, "${kafka.topic}");
|
||||
runner.setProperty(PutKafka.KEY, "${kafka.key}");
|
||||
runner.setProperty(PutKafka.TIMEOUT, "3 secs");
|
||||
runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue());
|
||||
@Test
|
||||
@Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...")
|
||||
public void testKeyValuePut() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(PutKafka.class);
|
||||
runner.setProperty(PutKafka.SEED_BROKERS, "192.168.0.101:9092");
|
||||
runner.setProperty(PutKafka.TOPIC, "${kafka.topic}");
|
||||
runner.setProperty(PutKafka.KEY, "${kafka.key}");
|
||||
runner.setProperty(PutKafka.TIMEOUT, "3 secs");
|
||||
runner.setProperty(PutKafka.DELIVERY_GUARANTEE, PutKafka.DELIVERY_REPLICATED.getValue());
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("kafka.topic", "test");
|
||||
attributes.put("kafka.key", "key3");
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("kafka.topic", "test");
|
||||
attributes.put("kafka.key", "key3");
|
||||
|
||||
final byte[] data = "Hello, World, Again! ;)".getBytes();
|
||||
runner.enqueue(data, attributes);
|
||||
runner.enqueue(data, attributes);
|
||||
runner.enqueue(data, attributes);
|
||||
runner.enqueue(data, attributes);
|
||||
final byte[] data = "Hello, World, Again! ;)".getBytes();
|
||||
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);
|
||||
final List<MockFlowFile> mffs = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS);
|
||||
final MockFlowFile mff = mffs.get(0);
|
||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 4);
|
||||
final List<MockFlowFile> mffs = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS);
|
||||
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 int failAfter = Integer.MAX_VALUE;
|
||||
private MockProducer producer;
|
||||
private int failAfter = Integer.MAX_VALUE;
|
||||
|
||||
public TestableProcessor() {
|
||||
}
|
||||
public TestableProcessor() {
|
||||
}
|
||||
|
||||
public TestableProcessor(final int failAfter) {
|
||||
this.failAfter = failAfter;
|
||||
}
|
||||
public TestableProcessor(final int failAfter) {
|
||||
this.failAfter = failAfter;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void instantiateProducer(final ProcessContext context) {
|
||||
producer = new MockProducer(createConfig(context));
|
||||
producer.setFailAfter(failAfter);
|
||||
}
|
||||
@OnScheduled
|
||||
public void instantiateProducer(final ProcessContext context) {
|
||||
producer = new MockProducer(createConfig(context));
|
||||
producer.setFailAfter(failAfter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
||||
return producer;
|
||||
}
|
||||
@Override
|
||||
protected Producer<byte[], byte[]> createProducer(final ProcessContext context) {
|
||||
return producer;
|
||||
}
|
||||
|
||||
public MockProducer getProducer() {
|
||||
return producer;
|
||||
}
|
||||
}
|
||||
public MockProducer getProducer() {
|
||||
return producer;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MockProducer extends Producer<byte[], byte[]> {
|
||||
|
||||
private static class MockProducer extends Producer<byte[], byte[]> {
|
||||
private int sendCount = 0;
|
||||
private int failAfter = Integer.MAX_VALUE;
|
||||
private int sendCount = 0;
|
||||
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) {
|
||||
super(config);
|
||||
|
@ -276,7 +277,7 @@ public class TestPutKafka {
|
|||
|
||||
@Override
|
||||
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"));
|
||||
} else {
|
||||
messages.add(message.message());
|
||||
|
@ -289,7 +290,7 @@ public class TestPutKafka {
|
|||
|
||||
@Override
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
@ -297,6 +298,6 @@ public class TestPutKafka {
|
|||
public void setFailAfter(final int successCount) {
|
||||
failAfter = successCount;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -26,12 +26,12 @@
|
|||
<module>nifi-kafka-nar</module>
|
||||
</modules>
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-processors</artifactId>
|
||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-processors</artifactId>
|
||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
</project>
|
||||
|
|
Loading…
Reference in New Issue