This commit is contained in:
joewitt 2015-04-25 09:20:35 -04:00
parent b2a1f5217d
commit 8f2502c4e4
7 changed files with 532 additions and 524 deletions

View File

@ -61,14 +61,18 @@ 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"),
@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")
.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)
@ -82,7 +86,8 @@ public class GetKafka extends AbstractProcessor {
.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")
.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)
@ -137,7 +142,6 @@ public class GetKafka extends AbstractProcessor {
.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;

View File

@ -61,12 +61,18 @@ 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")
@ -131,7 +137,6 @@ public class PutKafka extends AbstractProcessor {
.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")
@ -170,7 +175,6 @@ public class PutKafka extends AbstractProcessor {
return relationships;
}
@OnStopped
public void closeProducers() {
Producer<byte[], byte[]> producer;
@ -393,7 +397,8 @@ public class PutKafka extends AbstractProcessor {
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[] {
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);

View File

@ -37,7 +37,6 @@ import org.mockito.stubbing.Answer;
public class TestGetKafka {
@BeforeClass
public static void configureLogging() {
System.setProperty("org.slf4j.simpleLogger.log.kafka", "INFO");
@ -64,7 +63,6 @@ public class TestGetKafka {
}
}
@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;

View File

@ -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();
@ -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);
@ -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);
@ -235,8 +236,8 @@ public class TestPutKafka {
assertTrue(Arrays.equals(data, mff.toByteArray()));
}
private static class TestableProcessor extends PutKafka {
private MockProducer producer;
private int failAfter = Integer.MAX_VALUE;
@ -263,8 +264,8 @@ public class TestPutKafka {
}
}
private static class MockProducer extends Producer<byte[], byte[]> {
private int sendCount = 0;
private int failAfter = Integer.MAX_VALUE;