NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow batching of FlowFiles within a single publish and to let messages timeout if not acknowledged

Signed-off-by: Bryan Bende <bbende@apache.org>
This commit is contained in:
Mark Payne 2016-09-08 19:37:35 -04:00 committed by Bryan Bende
parent a4ed622152
commit b9cb6b1b47
No known key found for this signature in database
GPG Key ID: A0DDA9ED50711C39
45 changed files with 2834 additions and 3662 deletions

View File

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.stream.io.exception;
import java.io.IOException;
public class TokenTooLargeException extends IOException {
public TokenTooLargeException(final String message) {
super(message);
}
}

View File

@ -16,9 +16,12 @@
*/
package org.apache.nifi.stream.io.util;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.stream.io.exception.TokenTooLargeException;
/**
* The <code>StreamDemarcator</code> class takes an input stream and demarcates
* it so it could be read (see {@link #nextToken()}) as individual byte[]
@ -26,7 +29,7 @@ import java.io.InputStream;
* stream will be read into a single token which may result in
* {@link OutOfMemoryError} if stream is too large.
*/
public class StreamDemarcator {
public class StreamDemarcator implements Closeable {
private final static int INIT_BUFFER_SIZE = 8192;
@ -95,8 +98,10 @@ public class StreamDemarcator {
/**
* Will read the next data token from the {@link InputStream} returning null
* when it reaches the end of the stream.
*
* @throws IOException if unable to read from the stream
*/
public byte[] nextToken() {
public byte[] nextToken() throws IOException {
byte[] data = null;
int j = 0;
@ -126,8 +131,10 @@ public class StreamDemarcator {
/**
* Will fill the current buffer from current 'index' position, expanding it
* and or shuffling it if necessary
*
* @throws IOException if unable to read from the stream
*/
private void fill() {
private void fill() throws IOException {
if (this.index >= this.buffer.length) {
if (this.mark == 0) { // expand
byte[] newBuff = new byte[this.buffer.length + this.initialBufferSize];
@ -141,7 +148,6 @@ public class StreamDemarcator {
}
}
try {
int bytesRead;
do {
bytesRead = this.is.read(this.buffer, this.index, this.buffer.length - this.index);
@ -150,12 +156,9 @@ public class StreamDemarcator {
if (bytesRead != -1) {
this.readAheadLength = this.index + bytesRead;
if (this.readAheadLength > this.maxDataSize) {
throw new IllegalStateException("Maximum allowed data size of " + this.maxDataSize + " exceeded.");
throw new TokenTooLargeException("A message in the stream exceeds the maximum allowed message size of " + this.maxDataSize + " bytes.");
}
}
} catch (IOException e) {
throw new IllegalStateException("Failed while reading InputStream", e);
}
}
/**
@ -188,4 +191,9 @@ public class StreamDemarcator {
throw new IllegalArgumentException("'delimiterBytes' is an optional argument, but when provided its length must be > 0");
}
}
@Override
public void close() throws IOException {
is.close();
}
}

View File

@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
@ -65,7 +66,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateNoDelimiter() {
public void validateNoDelimiter() throws IOException {
String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, null, 1000);
@ -76,7 +77,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateNoDelimiterSmallInitialBuffer() {
public void validateNoDelimiterSmallInitialBuffer() throws IOException {
String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, null, 1000, 1);
@ -84,7 +85,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateSingleByteDelimiter() {
public void validateSingleByteDelimiter() throws IOException {
String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@ -95,7 +96,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateDelimiterAtTheBeginning() {
public void validateDelimiterAtTheBeginning() throws IOException {
String data = ",Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@ -106,7 +107,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateEmptyDelimiterSegments() {
public void validateEmptyDelimiterSegments() throws IOException {
String data = ",,,,,Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000);
@ -117,7 +118,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateSingleByteDelimiterSmallInitialBuffer() {
public void validateSingleByteDelimiterSmallInitialBuffer() throws IOException {
String data = "Learn from yesterday, live for today, hope for tomorrow. The important thing is not to stop questioning.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, ",".getBytes(StandardCharsets.UTF_8), 1000, 2);
@ -128,7 +129,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithMultiByteDelimiter() {
public void validateWithMultiByteDelimiter() throws IOException {
String data = "foodaabardaabazzz";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
@ -139,7 +140,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithMultiByteDelimiterAtTheBeginning() {
public void validateWithMultiByteDelimiterAtTheBeginning() throws IOException {
String data = "daafoodaabardaabazzz";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
@ -150,7 +151,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithMultiByteDelimiterSmallInitialBuffer() {
public void validateWithMultiByteDelimiterSmallInitialBuffer() throws IOException {
String data = "foodaabarffdaabazz";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000, 1);
@ -161,7 +162,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithMultiByteCharsNoDelimiter() {
public void validateWithMultiByteCharsNoDelimiter() throws IOException {
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, null, 1000);
@ -172,7 +173,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithMultiByteCharsNoDelimiterSmallInitialBuffer() {
public void validateWithMultiByteCharsNoDelimiterSmallInitialBuffer() throws IOException {
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
StreamDemarcator scanner = new StreamDemarcator(is, null, 1000, 2);
@ -183,7 +184,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateWithComplexDelimiter() {
public void validateWithComplexDelimiter() throws IOException {
String data = "THIS IS MY TEXT<MYDELIMITER>THIS IS MY NEW TEXT<MYDELIMITER>THIS IS MY NEWEST TEXT";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
StreamDemarcator scanner = new StreamDemarcator(is, "<MYDELIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
@ -193,8 +194,8 @@ public class StreamDemarcatorTest {
assertNull(scanner.nextToken());
}
@Test(expected = IllegalStateException.class)
public void validateMaxBufferSize() {
@Test(expected = IOException.class)
public void validateMaxBufferSize() throws IOException {
String data = "THIS IS MY TEXT<MY DELIMITER>THIS IS MY NEW TEXT<MY DELIMITER>THIS IS MY NEWEST TEXT";
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
StreamDemarcator scanner = new StreamDemarcator(is, "<MY DELIMITER>".getBytes(StandardCharsets.UTF_8), 20);
@ -202,7 +203,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateScannerHandlesNegativeOneByteInputsNoDelimiter() {
public void validateScannerHandlesNegativeOneByteInputsNoDelimiter() throws IOException {
ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
StreamDemarcator scanner = new StreamDemarcator(is, null, 20);
byte[] b = scanner.nextToken();
@ -210,7 +211,7 @@ public class StreamDemarcatorTest {
}
@Test
public void validateScannerHandlesNegativeOneByteInputs() {
public void validateScannerHandlesNegativeOneByteInputs() throws IOException {
ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
StreamDemarcator scanner = new StreamDemarcator(is, "water".getBytes(StandardCharsets.UTF_8), 20, 1024);
byte[] b = scanner.nextToken();
@ -218,10 +219,59 @@ public class StreamDemarcatorTest {
}
@Test
public void verifyScannerHandlesNegativeOneByteDelimiter() {
public void verifyScannerHandlesNegativeOneByteDelimiter() throws IOException {
ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
StreamDemarcator scanner = new StreamDemarcator(is, new byte[] { -1 }, 20, 1024);
assertArrayEquals(scanner.nextToken(), new byte[] { 0, 0, 0, 0 });
assertArrayEquals(scanner.nextToken(), new byte[] { 0, 0, 0 });
}
@Test
public void testWithoutTrailingDelimiter() throws IOException {
final byte[] inputData = "Larger Message First\nSmall".getBytes(StandardCharsets.UTF_8);
ByteArrayInputStream is = new ByteArrayInputStream(inputData);
StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000);
final byte[] first = scanner.nextToken();
final byte[] second = scanner.nextToken();
assertNotNull(first);
assertNotNull(second);
assertEquals("Larger Message First", new String(first, StandardCharsets.UTF_8));
assertEquals("Small", new String(second, StandardCharsets.UTF_8));
}
@Test
public void testOnBufferSplitNoTrailingDelimiter() throws IOException {
final byte[] inputData = "Yes\nNo".getBytes(StandardCharsets.UTF_8);
ByteArrayInputStream is = new ByteArrayInputStream(inputData);
StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000, 3);
final byte[] first = scanner.nextToken();
final byte[] second = scanner.nextToken();
assertNotNull(first);
assertNotNull(second);
assertArrayEquals(first, new byte[] {'Y', 'e', 's'});
assertArrayEquals(second, new byte[] {'N', 'o'});
}
@Test
public void testOnBufferSplit() throws IOException {
final byte[] inputData = "123\n456\n789".getBytes(StandardCharsets.UTF_8);
ByteArrayInputStream is = new ByteArrayInputStream(inputData);
StreamDemarcator scanner = new StreamDemarcator(is, "\n".getBytes(), 1000, 3);
final byte[] first = scanner.nextToken();
final byte[] second = scanner.nextToken();
final byte[] third = scanner.nextToken();
assertNotNull(first);
assertNotNull(second);
assertNotNull(third);
assertArrayEquals(first, new byte[] {'1', '2', '3'});
assertArrayEquals(second, new byte[] {'4', '5', '6'});
assertArrayEquals(third, new byte[] {'7', '8', '9'});
}
}

View File

@ -369,54 +369,55 @@ public class StandardProcessorTestRunner implements TestRunner {
}
@Override
public void enqueue(final Path path) throws IOException {
enqueue(path, new HashMap<String, String>());
public MockFlowFile enqueue(final Path path) throws IOException {
return enqueue(path, new HashMap<String, String>());
}
@Override
public void enqueue(final Path path, final Map<String, String> attributes) throws IOException {
public MockFlowFile enqueue(final Path path, final Map<String, String> attributes) throws IOException {
final Map<String, String> modifiedAttributes = new HashMap<>(attributes);
if (!modifiedAttributes.containsKey(CoreAttributes.FILENAME.key())) {
modifiedAttributes.put(CoreAttributes.FILENAME.key(), path.toFile().getName());
}
try (final InputStream in = Files.newInputStream(path)) {
enqueue(in, modifiedAttributes);
return enqueue(in, modifiedAttributes);
}
}
@Override
public void enqueue(final byte[] data) {
enqueue(data, new HashMap<String, String>());
public MockFlowFile enqueue(final byte[] data) {
return enqueue(data, new HashMap<String, String>());
}
@Override
public void enqueue(final String data) {
enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.emptyMap());
public MockFlowFile enqueue(final String data) {
return enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.<String, String> emptyMap());
}
@Override
public void enqueue(final byte[] data, final Map<String, String> attributes) {
enqueue(new ByteArrayInputStream(data), attributes);
public MockFlowFile enqueue(final byte[] data, final Map<String, String> attributes) {
return enqueue(new ByteArrayInputStream(data), attributes);
}
@Override
public void enqueue(final String data, final Map<String, String> attributes) {
enqueue(data.getBytes(StandardCharsets.UTF_8), attributes);
public MockFlowFile enqueue(final String data, final Map<String, String> attributes) {
return enqueue(data.getBytes(StandardCharsets.UTF_8), attributes);
}
@Override
public void enqueue(final InputStream data) {
enqueue(data, new HashMap<String, String>());
public MockFlowFile enqueue(final InputStream data) {
return enqueue(data, new HashMap<String, String>());
}
@Override
public void enqueue(final InputStream data, final Map<String, String> attributes) {
public MockFlowFile enqueue(final InputStream data, final Map<String, String> attributes) {
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor);
MockFlowFile flowFile = session.create();
flowFile = session.importFrom(data, flowFile);
flowFile = session.putAllAttributes(flowFile, attributes);
enqueue(flowFile);
return flowFile;
}
@Override
@ -878,17 +879,20 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void assertAllConditionsMet(final Relationship relationship, Predicate<MockFlowFile> predicate) {
if (predicate==null)
if (predicate==null) {
Assert.fail("predicate cannot be null");
}
final List<MockFlowFile> flowFiles = getFlowFilesForRelationship(relationship);
if (flowFiles.isEmpty())
if (flowFiles.isEmpty()) {
Assert.fail("Relationship " + relationship.getName() + " does not contain any FlowFile");
}
for (MockFlowFile flowFile : flowFiles) {
if (predicate.test(flowFile)==false)
if (predicate.test(flowFile)==false) {
Assert.fail("FlowFile " + flowFile + " does not meet all condition");
}
}
}
}

View File

@ -383,7 +383,7 @@ public interface TestRunner {
* @param path to read content from
* @throws IOException if unable to read content
*/
void enqueue(Path path) throws IOException;
MockFlowFile enqueue(Path path) throws IOException;
/**
* Reads the content from the given {@link Path} into memory and creates a
@ -394,7 +394,7 @@ public interface TestRunner {
* @param attributes attributes to use for new flow file
* @throws IOException if unable to read content
*/
void enqueue(Path path, Map<String, String> attributes) throws IOException;
MockFlowFile enqueue(Path path, Map<String, String> attributes) throws IOException;
/**
* Copies the content from the given byte array into memory and creates a
@ -403,7 +403,7 @@ public interface TestRunner {
*
* @param data to enqueue
*/
void enqueue(byte[] data);
MockFlowFile enqueue(byte[] data);
/**
* Creates a FlowFile with the content set to the given string (in UTF-8 format), with no attributes,
@ -411,7 +411,7 @@ public interface TestRunner {
*
* @param data to enqueue
*/
void enqueue(String data);
MockFlowFile enqueue(String data);
/**
* Copies the content from the given byte array into memory and creates a
@ -421,7 +421,7 @@ public interface TestRunner {
* @param data to enqueue
* @param attributes to use for enqueued item
*/
void enqueue(byte[] data, Map<String, String> attributes);
MockFlowFile enqueue(byte[] data, Map<String, String> attributes);
/**
* Creates a FlowFile with the content set to the given string (in UTF-8 format), with the given attributes,
@ -430,7 +430,7 @@ public interface TestRunner {
* @param data to enqueue
* @param attributes to use for enqueued item
*/
void enqueue(String data, Map<String, String> attributes);
MockFlowFile enqueue(String data, Map<String, String> attributes);
/**
* Reads the content from the given {@link InputStream} into memory and
@ -439,7 +439,7 @@ public interface TestRunner {
*
* @param data to source data from
*/
void enqueue(InputStream data);
MockFlowFile enqueue(InputStream data);
/**
* Reads the content from the given {@link InputStream} into memory and
@ -449,7 +449,7 @@ public interface TestRunner {
* @param data source of data
* @param attributes to use for flow files
*/
void enqueue(InputStream data, Map<String, String> attributes);
MockFlowFile enqueue(InputStream data, Map<String, String> attributes);
/**
* Copies the contents of the given {@link MockFlowFile} into a byte array

View File

@ -223,7 +223,7 @@ public class ConsumeKafka_0_10 extends AbstractProcessor {
final byte[] demarcator = context.getProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR).isSet()
? context.getProperty(ConsumeKafka_0_10.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
: null;
final Map<String, String> props = new HashMap<>();
final Map<String, Object> props = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());

View File

@ -74,7 +74,7 @@ public class ConsumerPool implements Closeable {
public ConsumerPool(
final int maxConcurrentLeases,
final byte[] demarcator,
final Map<String, String> kafkaProperties,
final Map<String, Object> kafkaProperties,
final List<String> topics,
final long maxWaitMillis,
final String keyEncoding,
@ -148,7 +148,7 @@ public class ConsumerPool implements Closeable {
});
}
private void closeConsumer(final Consumer consumer) {
private void closeConsumer(final Consumer<?, ?> consumer) {
consumerClosedCountRef.incrementAndGet();
try {
consumer.unsubscribe();

View File

@ -0,0 +1,178 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.nifi.flowfile.FlowFile;
public class InFlightMessageTracker {
private final ConcurrentMap<FlowFile, Counts> messageCountsByFlowFile = new ConcurrentHashMap<>();
private final ConcurrentMap<FlowFile, Exception> failures = new ConcurrentHashMap<>();
private final Object progressMutex = new Object();
public void incrementAcknowledgedCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
counter.incrementAcknowledgedCount();
synchronized (progressMutex) {
progressMutex.notify();
}
}
public int getAcknowledgedCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.get(flowFile);
return (counter == null) ? 0 : counter.getAcknowledgedCount();
}
public void incrementSentCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
counter.incrementSentCount();
}
public int getSentCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.get(flowFile);
return (counter == null) ? 0 : counter.getSentCount();
}
public void fail(final FlowFile flowFile, final Exception exception) {
failures.putIfAbsent(flowFile, exception);
synchronized (progressMutex) {
progressMutex.notify();
}
}
public Exception getFailure(final FlowFile flowFile) {
return failures.get(flowFile);
}
public boolean isFailed(final FlowFile flowFile) {
return getFailure(flowFile) != null;
}
public void reset() {
messageCountsByFlowFile.clear();
failures.clear();
}
public PublishResult failOutstanding(final Exception exception) {
messageCountsByFlowFile.keySet().stream()
.filter(ff -> !isComplete(ff))
.filter(ff -> !failures.containsKey(ff))
.forEach(ff -> failures.put(ff, exception));
return createPublishResult();
}
private boolean isComplete(final FlowFile flowFile) {
final Counts counts = messageCountsByFlowFile.get(flowFile);
if (counts.getAcknowledgedCount() == counts.getSentCount()) {
// all messages received successfully.
return true;
}
if (failures.containsKey(flowFile)) {
// FlowFile failed so is complete
return true;
}
return false;
}
private boolean isComplete() {
return messageCountsByFlowFile.keySet().stream()
.allMatch(flowFile -> isComplete(flowFile));
}
void awaitCompletion(final long millis) throws InterruptedException, TimeoutException {
final long startTime = System.nanoTime();
final long maxTime = startTime + TimeUnit.MILLISECONDS.toNanos(millis);
while (System.nanoTime() < maxTime) {
synchronized (progressMutex) {
if (isComplete()) {
return;
}
progressMutex.wait(millis);
}
}
throw new TimeoutException();
}
PublishResult createPublishResult() {
return new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
if (failures.isEmpty()) {
return messageCountsByFlowFile.keySet();
}
final Set<FlowFile> flowFiles = new HashSet<>(messageCountsByFlowFile.keySet());
flowFiles.removeAll(failures.keySet());
return flowFiles;
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return failures.keySet();
}
@Override
public int getSuccessfulMessageCount(final FlowFile flowFile) {
return getAcknowledgedCount(flowFile);
}
@Override
public Exception getReasonForFailure(final FlowFile flowFile) {
return getFailure(flowFile);
}
};
}
public static class Counts {
private final AtomicInteger sentCount = new AtomicInteger(0);
private final AtomicInteger acknowledgedCount = new AtomicInteger(0);
public void incrementSentCount() {
sentCount.incrementAndGet();
}
public void incrementAcknowledgedCount() {
acknowledgedCount.incrementAndGet();
}
public int getAcknowledgedCount() {
return acknowledgedCount.get();
}
public int getSentCount() {
return sentCount.get();
}
}
}

View File

@ -27,8 +27,9 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.SaslConfigs;
@ -186,7 +187,7 @@ final class KafkaProcessorUtils {
final Class<?> classType;
public KafkaConfigValidator(final Class classType) {
public KafkaConfigValidator(final Class<?> classType) {
this.classType = classType;
}
@ -211,7 +212,8 @@ final class KafkaProcessorUtils {
return builder.toString();
}
static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map<String, String> mapToPopulate) {
static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
// Translate SSLContext Service configuration into Kafka properties
@ -230,28 +232,33 @@ final class KafkaProcessorUtils {
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
}
}
String pName = propertyDescriptor.getName();
String pValue = propertyDescriptor.isExpressionLanguageSupported()
String propertyName = propertyDescriptor.getName();
String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
: context.getProperty(propertyDescriptor).getValue();
if (pValue != null) {
if (pName.endsWith(".ms")) { // kafka standard time notation
pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS));
if (propertyValue != null) {
// If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
// or the standard NiFi time period such as "5 secs"
if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
}
if (isStaticStringFieldNamePresent(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(pName, pValue);
if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(propertyName, propertyValue);
}
}
}
}
private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) {
private static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
}
private static Set<String> getPublicStaticStringFieldValues(final Class... classes) {
private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
final Set<String> strings = new HashSet<>();
for (final Class classType : classes) {
for (final Class<?> classType : classes) {
for (final Field field : classType.getDeclaredFields()) {
if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) {
try {

View File

@ -1,236 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.stream.io.util.StreamDemarcator;
/**
* Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
* with sending contents of the {@link FlowFile}s to Kafka.
*/
class KafkaPublisher implements Closeable {
private final Producer<byte[], byte[]> kafkaProducer;
private volatile long ackWaitTime = 30000;
private final ComponentLog componentLog;
private final int ackCheckSize;
KafkaPublisher(Properties kafkaProperties, ComponentLog componentLog) {
this(kafkaProperties, 100, componentLog);
}
/**
* Creates an instance of this class as well as the instance of the
* corresponding Kafka {@link KafkaProducer} using provided Kafka
* configuration properties.
*
* @param kafkaProperties instance of {@link Properties} used to bootstrap
* {@link KafkaProducer}
*/
KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) {
this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
this.ackCheckSize = ackCheckSize;
this.componentLog = componentLog;
}
/**
* Publishes messages to Kafka topic. It uses {@link StreamDemarcator} to
* determine how many messages to Kafka will be sent from a provided
* {@link InputStream} (see {@link PublishingContext#getContentStream()}).
* It supports two publishing modes:
* <ul>
* <li>Sending all messages constructed from
* {@link StreamDemarcator#nextToken()} operation.</li>
* <li>Sending only unacknowledged messages constructed from
* {@link StreamDemarcator#nextToken()} operation.</li>
* </ul>
* The unacknowledged messages are determined from the value of
* {@link PublishingContext#getLastAckedMessageIndex()}.
* <br>
* This method assumes content stream affinity where it is expected that the
* content stream that represents the same Kafka message(s) will remain the
* same across possible retries. This is required specifically for cases
* where delimiter is used and a single content stream may represent
* multiple Kafka messages. The
* {@link PublishingContext#getLastAckedMessageIndex()} will provide the
* index of the last ACKed message, so upon retry only messages with the
* higher index are sent.
*
* @param publishingContext instance of {@link PublishingContext} which hold
* context information about the message(s) to be sent.
* @return The index of the last successful offset.
*/
KafkaPublisherResult publish(PublishingContext publishingContext) {
StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());
int prevLastAckedMessageIndex = publishingContext.getLastAckedMessageIndex();
List<Future<RecordMetadata>> resultFutures = new ArrayList<>();
byte[] messageBytes;
int tokenCounter = 0;
boolean continueSending = true;
KafkaPublisherResult result = null;
for (; continueSending && (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) {
if (prevLastAckedMessageIndex < tokenCounter) {
ProducerRecord<byte[], byte[]> message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes);
resultFutures.add(this.kafkaProducer.send(message));
if (tokenCounter % this.ackCheckSize == 0) {
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
resultFutures.clear();
if (lastAckedMessageIndex % this.ackCheckSize != 0) {
continueSending = false;
result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
}
prevLastAckedMessageIndex = lastAckedMessageIndex;
}
}
}
if (result == null) {
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
resultFutures.clear();
result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
}
return result;
}
/**
* Sets the time this publisher will wait for the {@link Future#get()}
* operation (the Future returned by
* {@link KafkaProducer#send(ProducerRecord)}) to complete before timing
* out.
*
* This value will also be used as a timeout when closing the underlying
* {@link KafkaProducer}. See {@link #close()}.
*/
void setAckWaitTime(long ackWaitTime) {
this.ackWaitTime = ackWaitTime;
}
/**
* This operation will process ACKs from Kafka in the order in which
* {@link KafkaProducer#send(ProducerRecord)} invocation were made returning
* the index of the last ACKed message. Within this operation processing ACK
* simply means successful invocation of 'get()' operation on the
* {@link Future} returned by {@link KafkaProducer#send(ProducerRecord)}
* operation. Upon encountering any type of error while interrogating such
* {@link Future} the ACK loop will end. Messages that were not ACKed would
* be considered non-delivered and therefore could be resent at the later
* time.
*
* @param sendFutures list of {@link Future}s representing results of
* publishing to Kafka
*
* @param lastAckMessageIndex the index of the last ACKed message. It is
* important to provide the last ACKed message especially while re-trying so
* the proper index is maintained.
*/
private int processAcks(List<Future<RecordMetadata>> sendFutures, int lastAckMessageIndex) {
boolean exceptionThrown = false;
for (int segmentCounter = 0; segmentCounter < sendFutures.size() && !exceptionThrown; segmentCounter++) {
Future<RecordMetadata> future = sendFutures.get(segmentCounter);
try {
future.get(this.ackWaitTime, TimeUnit.MILLISECONDS);
lastAckMessageIndex++;
} catch (InterruptedException e) {
exceptionThrown = true;
Thread.currentThread().interrupt();
this.warnOrError("Interrupted while waiting for acks from Kafka", null);
} catch (ExecutionException e) {
exceptionThrown = true;
this.warnOrError("Failed while waiting for acks from Kafka", e);
} catch (TimeoutException e) {
exceptionThrown = true;
this.warnOrError("Timed out while waiting for acks from Kafka", null);
}
}
return lastAckMessageIndex;
}
/**
* Will close the underlying {@link KafkaProducer} waiting if necessary for
* the same duration as supplied {@link #setAckWaitTime(long)}
*/
@Override
public void close() {
this.kafkaProducer.close(this.ackWaitTime, TimeUnit.MILLISECONDS);
}
/**
*
*/
private void warnOrError(String message, Exception e) {
if (e == null) {
this.componentLog.warn(message);
} else {
this.componentLog.error(message, e);
}
}
/**
* Encapsulates the result received from publishing messages to Kafka
*/
static class KafkaPublisherResult {
private final int messagesSent;
private final int lastMessageAcked;
KafkaPublisherResult(int messagesSent, int lastMessageAcked) {
this.messagesSent = messagesSent;
this.lastMessageAcked = lastMessageAcked;
}
public int getMessagesSent() {
return this.messagesSent;
}
public int getLastMessageAcked() {
return this.lastMessageAcked;
}
public boolean isAllAcked() {
return this.lastMessageAcked > -1 && this.messagesSent - 1 == this.lastMessageAcked;
}
@Override
public String toString() {
return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
}
}
}

View File

@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
@ -27,17 +28,16 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import javax.xml.bind.DatatypeConverter;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnStopped;
@ -46,22 +46,18 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.FlowFileFilters;
import org.apache.nifi.processor.util.StandardValidators;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.10"})
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.10 producer. "
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.9.x"})
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.9 producer. "
+ "The messages to send may be individual FlowFiles or may be delimited, using a "
+ "user-specified delimiter, such as a new-line. "
+ " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring"
@ -72,20 +68,10 @@ import org.slf4j.LoggerFactory;
description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+ " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
private final Logger logger = LoggerFactory.getLogger(this.getClass());
protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id";
protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx";
protected static final String FAILED_TOPIC_ATTR = "failed.topic";
protected static final String FAILED_KEY_ATTR = "failed.key";
protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter";
@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
+ "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+ "be greater than 1.")
public class PublishKafka_0_10 extends AbstractProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -106,6 +92,10 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
static final AllowableValue RANDOM_PARTITIONING = new AllowableValue("org.apache.kafka.clients.producer.internals.DefaultPartitioner",
"DefaultPartitioner", "Messages will be assigned to random partitions.");
static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string.");
static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded",
"The key is interpreted as arbitrary binary data that is encoded using hexadecimal characters with uppercase letters.");
static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
.name("topic")
.displayName("Topic Name")
@ -125,15 +115,26 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
.defaultValue(DELIVERY_BEST_EFFORT.getValue())
.build();
static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder()
static final PropertyDescriptor METADATA_WAIT_TIME = new PropertyDescriptor.Builder()
.name(ProducerConfig.MAX_BLOCK_MS_CONFIG)
.displayName("Meta Data Wait Time")
.description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the "
.displayName("Max Metadata Wait Time")
.description("The amount of time publisher will wait to obtain metadata or wait for the buffer to flush during the 'send' call before failing the "
+ "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.defaultValue("30 sec")
.defaultValue("5 sec")
.build();
static final PropertyDescriptor ACK_WAIT_TIME = new PropertyDescriptor.Builder()
.name("ack.wait.time")
.displayName("Acknowledgment Wait Time")
.description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
+ "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(false)
.required(true)
.defaultValue("5 secs")
.build();
static final PropertyDescriptor MAX_REQUEST_SIZE = new PropertyDescriptor.Builder()
@ -148,10 +149,9 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
.name("kafka-key")
.displayName("Kafka Key")
.description("The Key to use for the Message. It will be serialized as UTF-8 bytes. "
+ "If not specified then the flow file attribute kafka.key is used if present "
+ "and we're not demarcating. In that case the hex string is coverted to its byte"
+ "form and written as a byte[] key.")
.description("The Key to use for the Message. "
+ "If not specified, the flow file attribute 'kafka.key' is used as the message key, if it is present "
+ "and we're not demarcating.")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
@ -175,7 +175,7 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
.description("Specifies the string (interpreted as UTF-8) to use for demarcating 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. "
+ "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on your OS.")
+ "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter, depending on your OS.")
.build();
static final PropertyDescriptor PARTITION_CLASS = new PropertyDescriptor.Builder()
@ -207,39 +207,31 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
.build();
static final List<PropertyDescriptor> DESCRIPTORS;
private static final List<PropertyDescriptor> PROPERTIES;
private static final Set<Relationship> RELATIONSHIPS;
static final Set<Relationship> RELATIONSHIPS;
private volatile PublisherPool publisherPool = null;
private volatile String brokers;
private final AtomicInteger taskCounter = new AtomicInteger();
private volatile boolean acceptTask = true;
/*
* Will ensure that list of PropertyDescriptors is build only once, since
* all other lifecycle methods are invoked multiple times.
*/
static {
final List<PropertyDescriptor> _descriptors = new ArrayList<>();
_descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
_descriptors.add(TOPIC);
_descriptors.add(DELIVERY_GUARANTEE);
_descriptors.add(KEY);
_descriptors.add(KEY_ATTRIBUTE_ENCODING);
_descriptors.add(MESSAGE_DEMARCATOR);
_descriptors.add(MAX_REQUEST_SIZE);
_descriptors.add(META_WAIT_TIME);
_descriptors.add(PARTITION_CLASS);
_descriptors.add(COMPRESSION_CODEC);
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
properties.add(TOPIC);
properties.add(DELIVERY_GUARANTEE);
properties.add(KEY);
properties.add(KEY_ATTRIBUTE_ENCODING);
properties.add(MESSAGE_DEMARCATOR);
properties.add(MAX_REQUEST_SIZE);
properties.add(ACK_WAIT_TIME);
properties.add(METADATA_WAIT_TIME);
properties.add(PARTITION_CLASS);
properties.add(COMPRESSION_CODEC);
DESCRIPTORS = Collections.unmodifiableList(_descriptors);
PROPERTIES = Collections.unmodifiableList(properties);
final Set<Relationship> _relationships = new HashSet<>();
_relationships.add(REL_SUCCESS);
_relationships.add(REL_FAILURE);
RELATIONSHIPS = Collections.unmodifiableSet(_relationships);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
RELATIONSHIPS = Collections.unmodifiableSet(relationships);
}
@Override
@ -249,14 +241,16 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return DESCRIPTORS;
return PROPERTIES;
}
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
.name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true)
.name(propertyDescriptorName)
.addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
.dynamic(true)
.build();
}
@ -265,226 +259,123 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
return KafkaProcessorUtils.validateCommonProperties(validationContext);
}
volatile KafkaPublisher kafkaPublisher;
/**
* This thread-safe operation will delegate to
* {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first
* checking and creating (if necessary) Kafka resource which could be either
* {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and
* destroy the underlying Kafka resource upon catching an {@link Exception}
* raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}.
* After Kafka resource is destroyed it will be re-created upon the next
* invocation of this operation essentially providing a self healing
* mechanism to deal with potentially corrupted resource.
* <p>
* Keep in mind that upon catching an exception the state of this processor
* will be set to no longer accept any more tasks, until Kafka resource is
* reset. This means that in a multi-threaded situation currently executing
* tasks will be given a chance to complete while no new tasks will be
* accepted.
*
* @param context context
* @param sessionFactory factory
*/
@Override
public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted.
this.taskCounter.incrementAndGet();
final ProcessSession session = sessionFactory.createSession();
try {
/*
* We can't be doing double null check here since as a pattern
* it only works for lazy init but not reset, which is what we
* are doing here. In fact the first null check is dangerous
* since 'kafkaPublisher' can become null right after its null
* check passed causing subsequent NPE.
*/
synchronized (this) {
if (this.kafkaPublisher == null) {
this.kafkaPublisher = this.buildKafkaResource(context, session);
}
private synchronized PublisherPool getPublisherPool(final ProcessContext context) {
PublisherPool pool = publisherPool;
if (pool != null) {
return pool;
}
/*
* The 'processed' boolean flag does not imply any failure or success. It simply states that:
* - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated
* - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile
*/
boolean processed = this.rendezvousWithKafka(context, session);
session.commit();
if (!processed) {
context.yield();
}
} catch (Throwable e) {
this.acceptTask = false;
session.rollback(true);
this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e});
} finally {
synchronized (this) {
if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) {
this.close();
this.acceptTask = true;
}
}
}
} else {
this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
context.yield();
}
return publisherPool = createPublisherPool(context);
}
protected PublisherPool createPublisherPool(final ProcessContext context) {
final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
final Map<String, Object> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis);
}
/**
* Will call {@link Closeable#close()} on the target resource after which
* the target resource will be set to null. Should only be called when there
* are no more threads being executed on this processor or when it has been
* verified that only a single thread remains.
*
* @see KafkaPublisher
* @see KafkaConsumer
*/
@OnStopped
public void close() {
try {
if (this.kafkaPublisher != null) {
try {
this.kafkaPublisher.close();
} catch (Exception e) {
this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e);
}
}
} finally {
this.kafkaPublisher = null;
}
public void closePool() {
if (publisherPool != null) {
publisherPool.close();
}
/**
* Will rendezvous with Kafka if {@link ProcessSession} contains
* {@link FlowFile} producing a result {@link FlowFile}.
* <br>
* The result {@link FlowFile} that is successful is then transfered to
* {@link #REL_SUCCESS}
* <br>
* The result {@link FlowFile} that is failed is then transfered to
* {@link #REL_FAILURE}
*
*/
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) {
FlowFile flowFile = session.get();
if (flowFile != null) {
long start = System.nanoTime();
flowFile = this.doRendezvousWithKafka(flowFile, context, session);
Relationship relationship = REL_SUCCESS;
if (!this.isFailedFlowFile(flowFile)) {
String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic);
session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration);
this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis",
new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration});
publisherPool = null;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet();
final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500));
if (flowFiles.isEmpty()) {
return;
}
final PublisherPool pool = getPublisherPool(context);
if (pool == null) {
context.yield();
return;
}
final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final long startTime = System.nanoTime();
try (final PublisherLease lease = pool.obtainPublisher()) {
// Send each FlowFile to Kafka asynchronously.
for (final FlowFile flowFile : flowFiles) {
if (!isScheduled()) {
// If stopped, re-queue FlowFile instead of sending it
session.transfer(flowFile);
continue;
}
final byte[] messageKey = getMessageKey(flowFile, context);
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
final byte[] demarcatorBytes;
if (useDemarcator) {
demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8);
} else {
relationship = REL_FAILURE;
flowFile = session.penalize(flowFile);
}
session.transfer(flowFile, relationship);
}
return flowFile != null;
demarcatorBytes = null;
}
/**
* Builds and instance of {@link KafkaPublisher}.
*/
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) {
final Map<String, String> kafkaProps = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps);
kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProps.put("max.request.size", String.valueOf(context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue()));
this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final Properties props = new Properties();
props.putAll(kafkaProps);
KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger());
return publisher;
}
/**
* Will rendezvous with {@link KafkaPublisher} after building
* {@link PublishingContext} and will produce the resulting
* {@link FlowFile}. The resulting FlowFile contains all required
* information to determine if message publishing originated from the
* provided FlowFile has actually succeeded fully, partially or failed
* completely (see {@link #isFailedFlowFile(FlowFile)}.
*/
private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
final AtomicReference<KafkaPublisher.KafkaPublisherResult> publishResultRef = new AtomicReference<>();
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(InputStream contentStream) throws IOException {
PublishingContext publishingContext = PublishKafka_0_10.this.buildPublishingContext(flowFile, context, contentStream);
KafkaPublisher.KafkaPublisherResult result = PublishKafka_0_10.this.kafkaPublisher.publish(publishingContext);
publishResultRef.set(result);
public void process(final InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
lease.publish(flowFile, in, messageKey, demarcatorBytes, topic);
}
}
});
FlowFile resultFile = publishResultRef.get().isAllAcked()
? this.cleanUpFlowFileIfNecessary(flowFile, session)
: session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context));
if (!this.isFailedFlowFile(resultFile)) {
resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent()));
}
return resultFile;
}
/**
* Builds {@link PublishingContext} for message(s) to be sent to Kafka.
* {@link PublishingContext} contains all contextual information required by
* {@link KafkaPublisher} to publish to Kafka. Such information contains
* things like topic name, content stream, delimiter, key and last ACKed
* message for cases where provided FlowFile is being retried (failed in the
* past).
* <br>
* For the clean FlowFile (file that has been sent for the first time),
* PublishingContext will be built form {@link ProcessContext} associated
* with this invocation.
* <br>
* For the failed FlowFile, {@link PublishingContext} will be built from
* attributes of that FlowFile which by then will already contain required
* information (e.g., topic, key, delimiter etc.). This is required to
* ensure the affinity of the retry in the even where processor
* configuration has changed. However keep in mind that failed FlowFile is
* only considered a failed FlowFile if it is being re-processed by the same
* processor (determined via {@link #FAILED_PROC_ID_ATTR}, see
* {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to
* another PublishKafka0_10 processor it is treated as a fresh FlowFile
* regardless if it has #FAILED* attributes set.
*/
private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) {
final byte[] keyBytes = getMessageKey(flowFile, context);
// Complete the send
final PublishResult publishResult = lease.complete();
final String topicName;
final byte[] delimiterBytes;
int lastAckedMessageIndex = -1;
if (this.isFailedFlowFile(flowFile)) {
lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
// Transfer any successful FlowFiles.
final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
for (FlowFile success : publishResult.getSuccessfulFlowFiles()) {
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue();
final int msgCount = publishResult.getSuccessfulMessageCount(success);
success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
session.adjustCounter("Messages Sent", msgCount, true);
final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
session.transfer(success, REL_SUCCESS);
}
// Transfer any failures.
for (final FlowFile failure : publishResult.getFailedFlowFiles()) {
final int successCount = publishResult.getSuccessfulMessageCount(failure);
if (successCount > 0) {
getLogger().error("Failed to send some messages for {} to Kafka, but {} messages were acknowledged by Kafka. Routing to failure due to {}",
new Object[] {failure, successCount, publishResult.getReasonForFailure(failure)});
} else {
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR)
.evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
getLogger().error("Failed to send all message for {} to Kafka; routing to failure due to {}",
new Object[] {failure, publishResult.getReasonForFailure(failure)});
}
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex,
context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue());
publishingContext.setKeyBytes(keyBytes);
publishingContext.setDelimiterBytes(delimiterBytes);
return publishingContext;
session.transfer(failure, REL_FAILURE);
}
}
}
private byte[] getMessageKey(final FlowFile flowFile, final ProcessContext context) {
if (context.getProperty(MESSAGE_DEMARCATOR).isSet()) {
return null;
}
final String uninterpretedKey;
if (context.getProperty(KEY).isSet()) {
uninterpretedKey = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
@ -503,51 +394,4 @@ public class PublishKafka_0_10 extends AbstractSessionFactoryProcessor {
return DatatypeConverter.parseHexBinary(uninterpretedKey);
}
/**
* Will remove FAILED_* attributes if FlowFile is no longer considered a
* failed FlowFile
*
* @see #isFailedFlowFile(FlowFile)
*/
private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) {
if (this.isFailedFlowFile(flowFile)) {
Set<String> keysToRemove = new HashSet<>();
keysToRemove.add(FAILED_DELIMITER_ATTR);
keysToRemove.add(FAILED_KEY_ATTR);
keysToRemove.add(FAILED_TOPIC_ATTR);
keysToRemove.add(FAILED_PROC_ID_ATTR);
keysToRemove.add(FAILED_LAST_ACK_IDX);
flowFile = session.removeAllAttributes(flowFile, keysToRemove);
}
return flowFile;
}
/**
* Builds a {@link Map} of FAILED_* attributes
*
* @see #FAILED_PROC_ID_ATTR
* @see #FAILED_LAST_ACK_IDX
* @see #FAILED_TOPIC_ATTR
* @see #FAILED_KEY_ATTR
* @see #FAILED_DELIMITER_ATTR
*/
private Map<String, String> buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile, ProcessContext context) {
Map<String, String> attributes = new HashMap<>();
attributes.put(FAILED_PROC_ID_ATTR, this.getIdentifier());
attributes.put(FAILED_LAST_ACK_IDX, String.valueOf(lastAckedMessageIndex));
attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue());
attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue());
attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet()
? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null);
return attributes;
}
/**
* Returns 'true' if provided FlowFile is a failed FlowFile. A failed
* FlowFile contains {@link #FAILED_PROC_ID_ATTR}.
*/
private boolean isFailedFlowFile(FlowFile flowFile) {
return this.getIdentifier().equals(flowFile.getAttribute(FAILED_PROC_ID_ATTR));
}
}

View File

@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.Collection;
import java.util.Collections;
import org.apache.nifi.flowfile.FlowFile;
public interface PublishResult {
Collection<FlowFile> getSuccessfulFlowFiles();
Collection<FlowFile> getFailedFlowFiles();
int getSuccessfulMessageCount(FlowFile flowFile);
Exception getReasonForFailure(FlowFile flowFile);
public static final PublishResult EMPTY = new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
return Collections.emptyList();
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return Collections.emptyList();
}
@Override
public int getSuccessfulMessageCount(FlowFile flowFile) {
return 0;
}
@Override
public Exception getReasonForFailure(FlowFile flowFile) {
return null;
}
};
}

View File

@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.stream.io.exception.TokenTooLargeException;
import org.apache.nifi.stream.io.util.StreamDemarcator;
public class PublisherLease implements Closeable {
private final ComponentLog logger;
private final Producer<byte[], byte[]> producer;
private final int maxMessageSize;
private final long maxAckWaitMillis;
private volatile boolean poisoned = false;
private InFlightMessageTracker tracker;
public PublisherLease(final Producer<byte[], byte[]> producer, final int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
this.producer = producer;
this.maxMessageSize = maxMessageSize;
this.logger = logger;
this.maxAckWaitMillis = maxAckWaitMillis;
}
protected void poison() {
this.poisoned = true;
}
public boolean isPoisoned() {
return poisoned;
}
void publish(final FlowFile flowFile, final InputStream flowFileContent, final byte[] messageKey, final byte[] demarcatorBytes, final String topic) throws IOException {
if (tracker == null) {
tracker = new InFlightMessageTracker();
}
try (final StreamDemarcator demarcator = new StreamDemarcator(flowFileContent, demarcatorBytes, maxMessageSize)) {
byte[] messageContent;
try {
while ((messageContent = demarcator.nextToken()) != null) {
// We do not want to use any key if we have a demarcator because that would result in
// the key being the same for multiple messages
final byte[] keyToUse = demarcatorBytes == null ? messageKey : null;
publish(flowFile, keyToUse, messageContent, topic, tracker);
if (tracker.isFailed(flowFile)) {
// If we have a failure, don't try to send anything else.
return;
}
}
} catch (final TokenTooLargeException ttle) {
tracker.fail(flowFile, ttle);
}
} catch (final Exception e) {
tracker.fail(flowFile, e);
poison();
throw e;
}
}
private void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker) {
final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, null, messageKey, messageContent);
producer.send(record, new Callback() {
@Override
public void onCompletion(final RecordMetadata metadata, final Exception exception) {
if (exception == null) {
tracker.incrementAcknowledgedCount(flowFile);
} else {
tracker.fail(flowFile, exception);
poison();
}
}
});
tracker.incrementSentCount(flowFile);
}
public PublishResult complete() {
if (tracker == null) {
throw new IllegalStateException("Cannot complete publishing to Kafka because Publisher Lease was already closed");
}
producer.flush();
try {
tracker.awaitCompletion(maxAckWaitMillis);
return tracker.createPublishResult();
} catch (final InterruptedException e) {
logger.warn("Interrupted while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka");
Thread.currentThread().interrupt();
return tracker.failOutstanding(e);
} catch (final TimeoutException e) {
logger.warn("Timed out while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka");
return tracker.failOutstanding(e);
} finally {
tracker = null;
}
}
@Override
public void close() {
producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS);
tracker = null;
}
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.nifi.logging.ComponentLog;
public class PublisherPool implements Closeable {
private final ComponentLog logger;
private final BlockingQueue<PublisherLease> publisherQueue;
private final Map<String, Object> kafkaProperties;
private final int maxMessageSize;
private final long maxAckWaitMillis;
private volatile boolean closed = false;
PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis) {
this.logger = logger;
this.publisherQueue = new LinkedBlockingQueue<>();
this.kafkaProperties = kafkaProperties;
this.maxMessageSize = maxMessageSize;
this.maxAckWaitMillis = maxAckWaitMillis;
}
public PublisherLease obtainPublisher() {
if (isClosed()) {
throw new IllegalStateException("Connection Pool is closed");
}
PublisherLease lease = publisherQueue.poll();
if (lease != null) {
return lease;
}
lease = createLease();
return lease;
}
private PublisherLease createLease() {
final Producer<byte[], byte[]> producer = new KafkaProducer<>(kafkaProperties);
final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger) {
@Override
public void close() {
if (isPoisoned() || isClosed()) {
super.close();
} else {
publisherQueue.offer(this);
}
}
};
return lease;
}
public synchronized boolean isClosed() {
return closed;
}
@Override
public synchronized void close() {
closed = true;
PublisherLease lease;
while ((lease = publisherQueue.poll()) != null) {
lease.close();
}
}
/**
* Returns the number of leases that are currently available
*
* @return the number of leases currently available
*/
protected int available() {
return publisherQueue.size();
}
}

View File

@ -1,124 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
/**
* Holder of context information used by {@link KafkaPublisher} required to
* publish messages to Kafka.
*/
class PublishingContext {
private final InputStream contentStream;
private final String topic;
private final int lastAckedMessageIndex;
private final int maxRequestSize;
private byte[] keyBytes;
private byte[] delimiterBytes;
PublishingContext(InputStream contentStream, String topic) {
this(contentStream, topic, -1);
}
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
this(contentStream, topic, lastAckedMessageIndex, 1048576);
}
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex, int maxRequestSize) {
this.validateInput(contentStream, topic, lastAckedMessageIndex);
this.contentStream = contentStream;
this.topic = topic;
this.lastAckedMessageIndex = lastAckedMessageIndex;
this.maxRequestSize = maxRequestSize;
}
@Override
public String toString() {
return "topic: '" + this.topic + "'; delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'";
}
int getLastAckedMessageIndex() {
return this.lastAckedMessageIndex;
}
int getMaxRequestSize() {
return this.maxRequestSize;
}
byte[] getKeyBytes() {
return this.keyBytes;
}
byte[] getDelimiterBytes() {
return this.delimiterBytes;
}
InputStream getContentStream() {
return this.contentStream;
}
String getTopic() {
return this.topic;
}
void setKeyBytes(byte[] keyBytes) {
if (this.keyBytes == null) {
if (keyBytes != null) {
this.assertBytesValid(keyBytes);
this.keyBytes = keyBytes;
}
} else {
throw new IllegalArgumentException("'keyBytes' can only be set once per instance");
}
}
void setDelimiterBytes(byte[] delimiterBytes) {
if (this.delimiterBytes == null) {
if (delimiterBytes != null) {
this.assertBytesValid(delimiterBytes);
this.delimiterBytes = delimiterBytes;
}
} else {
throw new IllegalArgumentException("'delimiterBytes' can only be set once per instance");
}
}
private void assertBytesValid(byte[] bytes) {
if (bytes != null) {
if (bytes.length == 0) {
throw new IllegalArgumentException("'bytes' must not be empty");
}
}
}
private void validateInput(InputStream contentStream, String topic, int lastAckedMessageIndex) {
if (contentStream == null) {
throw new IllegalArgumentException("'contentStream' must not be null");
} else if (topic == null || topic.trim().length() == 0) {
throw new IllegalArgumentException("'topic' must not be null or empty");
} else if (lastAckedMessageIndex < -1) {
throw new IllegalArgumentException("'lastAckedMessageIndex' must be >= -1");
}
}
}

View File

@ -1,306 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult;
import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import kafka.consumer.Consumer;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.ConsumerTimeoutException;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import org.apache.kafka.clients.producer.ProducerConfig;
public class KafkaPublisherTest {
private static EmbeddedKafka kafkaLocal;
private static EmbeddedKafkaProducerHelper producerHelper;
@BeforeClass
public static void beforeClass() {
kafkaLocal = new EmbeddedKafka();
kafkaLocal.start();
producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal);
}
@AfterClass
public static void afterClass() throws Exception {
producerHelper.close();
kafkaLocal.stop();
}
@Test
public void validateSuccessfulSendAsWhole() throws Exception {
InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
String topicName = "validateSuccessfulSendAsWhole";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
KafkaPublisherResult result = publisher.publish(publishingContext);
assertEquals(0, result.getLastMessageAcked());
assertEquals(1, result.getMessagesSent());
contentStream.close();
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
assertNotNull(iter.next());
try {
iter.next();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
}
@Test
public void validateSuccessfulSendAsDelimited() throws Exception {
InputStream contentStream = new ByteArrayInputStream(
"Hello Kafka\nHello Kafka\nHello Kafka\nHello Kafka\n".getBytes(StandardCharsets.UTF_8));
String topicName = "validateSuccessfulSendAsDelimited";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
KafkaPublisherResult result = publisher.publish(publishingContext);
assertEquals(3, result.getLastMessageAcked());
assertEquals(4, result.getMessagesSent());
contentStream.close();
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
assertNotNull(iter.next());
assertNotNull(iter.next());
assertNotNull(iter.next());
assertNotNull(iter.next());
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
}
/*
* This test simulates the condition where not all messages were ACKed by
* Kafka
*/
@Test
public void validateRetries() throws Exception {
byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
InputStream contentStream = new ByteArrayInputStream(testValue);
String topicName = "validateSuccessfulReSendOfFailedSegments";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
// simulates the first re-try
int lastAckedMessageIndex = 1;
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
String m1 = new String(iter.next().message());
String m2 = new String(iter.next().message());
assertEquals("Hello Kafka3", m1);
assertEquals("Hello Kafka4", m2);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
// simulates the second re-try
lastAckedMessageIndex = 2;
contentStream = new ByteArrayInputStream(testValue);
publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
m1 = new String(iter.next().message());
assertEquals("Hello Kafka4", m1);
publisher.close();
}
/*
* Similar to the above test, but it sets the first retry index to the last
* possible message index and second index to an out of bound index. The
* expectation is that no messages will be sent to Kafka
*/
@Test
public void validateRetriesWithWrongIndex() throws Exception {
byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
InputStream contentStream = new ByteArrayInputStream(testValue);
String topicName = "validateRetriesWithWrongIndex";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
// simulates the first re-try
int lastAckedMessageIndex = 3;
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
// simulates the second re-try
lastAckedMessageIndex = 6;
contentStream = new ByteArrayInputStream(testValue);
publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
publisher.close();
}
@Test
public void validateWithMultiByteCharactersNoDelimiter() throws Exception {
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
String topicName = "validateWithMultiByteCharacters";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publisher.publish(publishingContext);
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
String r = new String(iter.next().message(), StandardCharsets.UTF_8);
assertEquals(data, r);
}
@Test
public void validateWithNonDefaultPartitioner() throws Exception {
String data = "fooandbarandbaz";
InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
String topicName = "validateWithNonDefaultPartitioner";
Properties kafkaProperties = this.buildProducerProperties();
kafkaProperties.setProperty("partitioner.class", TestPartitioner.class.getName());
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publishingContext.setDelimiterBytes("and".getBytes(StandardCharsets.UTF_8));
try {
publisher.publish(publishingContext);
// partitioner should be invoked 3 times
assertTrue(TestPartitioner.counter == 3);
publisher.close();
} finally {
TestPartitioner.counter = 0;
}
}
private Properties buildProducerProperties() {
Properties kafkaProperties = new Properties();
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort());
kafkaProperties.put("auto.create.topics.enable", "true");
return kafkaProperties;
}
private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
Properties props = new Properties();
props.put("zookeeper.connect", "localhost:" + kafkaLocal.getZookeeperPort());
props.put("group.id", "test");
props.put("consumer.timeout.ms", "500");
props.put("auto.offset.reset", "smallest");
ConsumerConfig consumerConfig = new ConsumerConfig(props);
ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig);
Map<String, Integer> topicCountMap = new HashMap<>(1);
topicCountMap.put(topic, 1);
Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
ConsumerIterator<byte[], byte[]> iter = streams.get(0).iterator();
return iter;
}
public static class TestPartitioner implements Partitioner {
static int counter;
@Override
public void configure(Map<String, ?> configs) {
// nothing to do, test
}
@Override
public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes,
Cluster cluster) {
counter++;
return 0;
}
@Override
public void close() {
counter = 0;
}
}
}

View File

@ -1,375 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.times;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.verify;
public class PublishKafkaTest {
@Test
public void validateCustomSerilaizerDeserializerSettings() throws Exception {
PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(PublishKafka_0_10.TOPIC, "foo");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3 sec");
runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
runner.assertValid();
runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo");
runner.assertNotValid();
}
@Test
public void validatePropertiesValidation() throws Exception {
PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(PublishKafka_0_10.TOPIC, "foo");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "foo");
try {
runner.assertValid();
fail();
} catch (AssertionError e) {
assertTrue(e.getMessage().contains("'max.block.ms' validated against 'foo' is invalid"));
}
}
@Test
public void validateCustomValidation() {
String topicName = "validateCustomValidation";
PublishKafka_0_10 publishKafka = new PublishKafka_0_10();
/*
* Validates that Kerberos principle is required if one of SASL set for
* secirity protocol
*/
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
try {
runner.run();
fail();
} catch (Throwable e) {
assertTrue(e.getMessage().contains("'Kerberos Service Name' is invalid because"));
}
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateSingleCharacterDemarcatedMessages() {
String topicName = "validateSingleCharacterDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerA() {
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo");
runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerB() {
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
StubPublishKafka putKafka = new StubPublishKafka(1);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "foo");
runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnSendFailureAndThenResendSuccessA() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "3000 millis");
final String text = "Hello World\nGoodbye\nfail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
putKafka.destroy();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnSendFailureAndThenResendSuccessB() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(1);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
final String text = "Hello World\nGoodbye\nfail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnFutureGetFailureAndThenResendSuccessFirstMessageFail() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
final String text = "futurefail\nHello World\nGoodbye\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0);
assertNotNull(ff);
runner.enqueue(ff);
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
// 6 sends due to duplication
verify(producer, times(5)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnFutureGetFailureAndThenResendSuccess() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka_0_10.META_WAIT_TIME, "500 millis");
final String text = "Hello World\nGoodbye\nfuturefail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).get(0);
assertNotNull(ff);
runner.enqueue(ff);
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
// 6 sends due to duplication
verify(producer, times(6)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateDemarcationIntoEmptyMessages() {
String topicName = "validateDemarcationIntoEmptyMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
final TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(PublishKafka_0_10.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "\n");
final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
runner.enqueue(bytes);
runner.run(1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexRightPartialDemarcatedMessages() {
String topicName = "validateComplexRightPartialDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(3)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexLeftPartialDemarcatedMessages() {
String topicName = "validateComplexLeftPartialDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>僠<僠WILDSTUFF僠>僠".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexPartialMatchDemarcatedMessages() {
String topicName = "validateComplexPartialMatchDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDBOOMSTUFF僠>僠".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(2)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@Test
public void validateUtf8Key() {
String topicName = "validateUtf8Key";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.KEY, "${myKey}");
final Map<String, String> attributes = Collections.singletonMap("myKey", "key1");
runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1);
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
final Map<Object, Object> msgs = putKafka.getMessagesSent();
assertEquals(1, msgs.size());
final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
assertTrue(Arrays.equals("key1".getBytes(StandardCharsets.UTF_8), msgKey));
}
@Test
public void validateHexKey() {
String topicName = "validateUtf8Key";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka_0_10.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka_0_10.KEY_ATTRIBUTE_ENCODING, KafkaProcessorUtils.HEX_ENCODING);
runner.setProperty(PublishKafka_0_10.KEY, "${myKey}");
final Map<String, String> attributes = Collections.singletonMap("myKey", "6B657931");
runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1);
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
final Map<Object, Object> msgs = putKafka.getMessagesSent();
assertEquals(1, msgs.size());
final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
assertTrue(Arrays.equals(new byte[] {0x6B, 0x65, 0x79, 0x31}, msgKey));
}
}

View File

@ -1,91 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import org.junit.Test;
public class PublishingContextTest {
@Test
public void failInvalidConstructorArgs() {
try {
new PublishingContext(null, null);
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), null);
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), "");
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), "mytopic", -3);
fail();
} catch (IllegalArgumentException e) {
// success
}
}
@Test
public void validateFullSetting() {
PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic", 3);
publishingContext.setDelimiterBytes("delimiter".getBytes(StandardCharsets.UTF_8));
publishingContext.setKeyBytes("key".getBytes(StandardCharsets.UTF_8));
assertEquals("delimiter", new String(publishingContext.getDelimiterBytes(), StandardCharsets.UTF_8));
assertEquals("key", new String(publishingContext.getKeyBytes(), StandardCharsets.UTF_8));
assertEquals("topic", publishingContext.getTopic());
assertEquals("topic: 'topic'; delimiter: 'delimiter'", publishingContext.toString());
}
@Test
public void validateOnlyOnceSetPerInstance() {
PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic");
publishingContext.setKeyBytes(new byte[]{0});
try {
publishingContext.setKeyBytes(new byte[]{0});
fail();
} catch (IllegalArgumentException e) {
// success
}
publishingContext.setDelimiterBytes(new byte[]{0});
try {
publishingContext.setDelimiterBytes(new byte[]{0});
fail();
} catch (IllegalArgumentException e) {
// success
}
}
}

View File

@ -1,143 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.lang.reflect.Field;
import static org.mockito.Mockito.when;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS;
import org.mockito.Mockito;
import static org.mockito.Mockito.mock;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class StubPublishKafka extends PublishKafka_0_10 {
private volatile Producer<byte[], byte[]> producer;
private volatile boolean failed;
private final int ackCheckSize;
private final ExecutorService executor = Executors.newCachedThreadPool();
private final Map<Object, Object> msgsSent = new ConcurrentHashMap<>();
StubPublishKafka(int ackCheckSize) {
this.ackCheckSize = ackCheckSize;
}
public Producer<byte[], byte[]> getProducer() {
return producer;
}
public void destroy() {
this.executor.shutdownNow();
}
public Map<Object, Object> getMessagesSent() {
return new HashMap<>(msgsSent);
}
@SuppressWarnings("unchecked")
@Override
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
throws ProcessException {
final Map<String, String> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
KafkaPublisher publisher;
try {
Field f = PublishKafka_0_10.class.getDeclaredField("brokers");
f.setAccessible(true);
f.set(this, context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue());
publisher = (KafkaPublisher) TestUtils.getUnsafe().allocateInstance(KafkaPublisher.class);
publisher.setAckWaitTime(15000);
producer = mock(Producer.class);
this.instrumentProducer(producer, false);
Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer");
kf.setAccessible(true);
kf.set(publisher, producer);
Field componentLogF = KafkaPublisher.class.getDeclaredField("componentLog");
componentLogF.setAccessible(true);
componentLogF.set(publisher, mock(ComponentLog.class));
Field ackCheckSizeField = KafkaPublisher.class.getDeclaredField("ackCheckSize");
ackCheckSizeField.setAccessible(true);
ackCheckSizeField.set(publisher, this.ackCheckSize);
} catch (Exception e) {
e.printStackTrace();
throw new IllegalStateException(e);
}
return publisher;
}
@SuppressWarnings("unchecked")
private void instrumentProducer(Producer<byte[], byte[]> producer, boolean failRandomly) {
when(producer.send(Mockito.any(ProducerRecord.class))).then(new Answer<Future<RecordMetadata>>() {
@Override
public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
if (record != null && record.key() != null) {
msgsSent.put(record.key(), record.value());
}
String value = new String(record.value(), StandardCharsets.UTF_8);
if ("fail".equals(value) && !StubPublishKafka.this.failed) {
StubPublishKafka.this.failed = true;
throw new RuntimeException("intentional");
}
Future<RecordMetadata> future = executor.submit(new Callable<RecordMetadata>() {
@Override
public RecordMetadata call() throws Exception {
if ("futurefail".equals(value) && !StubPublishKafka.this.failed) {
StubPublishKafka.this.failed = true;
throw new TopicAuthorizationException("Unauthorized");
} else {
TopicPartition partition = new TopicPartition("foo", 0);
RecordMetadata meta = new RecordMetadata(partition, 0, 0);
return meta;
}
}
});
return future;
}
});
}
}

View File

@ -0,0 +1,87 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import org.apache.nifi.util.MockFlowFile;
import org.junit.Assert;
import org.junit.Test;
public class TestInFlightMessageTracker {
@Test(timeout = 5000L)
public void testAwaitCompletionWhenComplete() throws InterruptedException, TimeoutException {
final MockFlowFile flowFile = new MockFlowFile(1L);
final InFlightMessageTracker tracker = new InFlightMessageTracker();
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementAcknowledgedCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementAcknowledgedCount(flowFile);
tracker.awaitCompletion(1L);
}
@Test(timeout = 5000L)
public void testAwaitCompletionWhileWaiting() throws InterruptedException, ExecutionException {
final MockFlowFile flowFile = new MockFlowFile(1L);
final InFlightMessageTracker tracker = new InFlightMessageTracker();
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
final ExecutorService exec = Executors.newFixedThreadPool(1);
final Future<?> future = exec.submit(() -> {
try {
tracker.awaitCompletion(10000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
tracker.incrementAcknowledgedCount(flowFile);
tracker.incrementAcknowledgedCount(flowFile);
future.get();
}
private void verifyNotComplete(final InFlightMessageTracker tracker) throws InterruptedException {
try {
tracker.awaitCompletion(10L);
Assert.fail("Expected timeout");
} catch (final TimeoutException te) {
// expected
}
}
}

View File

@ -0,0 +1,262 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Before;
import org.junit.Test;
public class TestPublishKafka {
private static final String TOPIC_NAME = "unit-test";
private PublisherPool mockPool;
private PublisherLease mockLease;
private TestRunner runner;
@Before
public void setup() {
mockPool = mock(PublisherPool.class);
mockLease = mock(PublisherLease.class);
when(mockPool.obtainPublisher()).thenReturn(mockLease);
runner = TestRunners.newTestRunner(new PublishKafka_0_10() {
@Override
protected PublisherPool createPublisherPool(final ProcessContext context) {
return mockPool;
}
});
runner.setProperty(PublishKafka_0_10.TOPIC, TOPIC_NAME);
}
@Test
public void testSingleSuccess() throws IOException {
final MockFlowFile flowFile = runner.enqueue("hello world");
when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFile, 1));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1);
verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleSuccess() throws IOException {
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFiles, 1));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 3);
verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
}
@Test
public void testSingleFailure() throws IOException {
final MockFlowFile flowFile = runner.enqueue("hello world");
when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_FAILURE, 1);
verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleFailures() throws IOException {
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFiles));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_FAILURE, 3);
verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleMessagesPerFlowFile() throws IOException {
final List<FlowFile> flowFiles = new ArrayList<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
msgCounts.put(flowFiles.get(0), 10);
msgCounts.put(flowFiles.get(1), 20);
final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap());
when(mockLease.complete()).thenReturn(result);
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 2);
verify(mockLease, times(2)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
runner.assertAllFlowFilesContainAttribute("msg.count");
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
.filter(ff -> ff.getAttribute("msg.count").equals("10"))
.count());
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
.filter(ff -> ff.getAttribute("msg.count").equals("20"))
.count());
}
@Test
public void testSomeSuccessSomeFailure() throws IOException {
final List<FlowFile> flowFiles = new ArrayList<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
msgCounts.put(flowFiles.get(0), 10);
msgCounts.put(flowFiles.get(1), 20);
final Map<FlowFile, Exception> failureMap = new HashMap<>();
failureMap.put(flowFiles.get(2), new RuntimeException("Intentional Unit Test Exception"));
failureMap.put(flowFiles.get(3), new RuntimeException("Intentional Unit Test Exception"));
final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles.subList(0, 2)), failureMap);
when(mockLease.complete()).thenReturn(result);
runner.run();
runner.assertTransferCount(PublishKafka_0_10.REL_SUCCESS, 2);
runner.assertTransferCount(PublishKafka_0_10.REL_FAILURE, 2);
verify(mockLease, times(4)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
.filter(ff -> "10".equals(ff.getAttribute("msg.count")))
.count());
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream()
.filter(ff -> "20".equals(ff.getAttribute("msg.count")))
.count());
assertTrue(runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).stream()
.noneMatch(ff -> ff.getAttribute("msg.count") != null));
}
private PublishResult createAllSuccessPublishResult(final FlowFile successfulFlowFile, final int msgCount) {
return createAllSuccessPublishResult(Collections.singleton(successfulFlowFile), msgCount);
}
private PublishResult createAllSuccessPublishResult(final Set<FlowFile> successfulFlowFiles, final int msgCountPerFlowFile) {
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
for (final FlowFile ff : successfulFlowFiles) {
msgCounts.put(ff, msgCountPerFlowFile);
}
return createPublishResult(msgCounts, successfulFlowFiles, Collections.emptyMap());
}
private PublishResult createFailurePublishResult(final FlowFile failure) {
return createFailurePublishResult(Collections.singleton(failure));
}
private PublishResult createFailurePublishResult(final Set<FlowFile> failures) {
final Map<FlowFile, Exception> failureMap = failures.stream().collect(Collectors.toMap(ff -> ff, ff -> new RuntimeException("Intentional Unit Test Exception")));
return createPublishResult(Collections.emptyMap(), Collections.emptySet(), failureMap);
}
private PublishResult createPublishResult(final Map<FlowFile, Integer> msgCounts, final Set<FlowFile> successFlowFiles, final Map<FlowFile, Exception> failures) {
// sanity check.
for (final FlowFile success : successFlowFiles) {
if (failures.containsKey(success)) {
throw new IllegalArgumentException("Found same FlowFile in both 'success' and 'failures' collections: " + success);
}
}
return new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
return successFlowFiles;
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return failures.keySet();
}
@Override
public int getSuccessfulMessageCount(FlowFile flowFile) {
Integer count = msgCounts.get(flowFile);
return count == null ? 0 : count.intValue();
}
@Override
public Exception getReasonForFailure(FlowFile flowFile) {
return failures.get(flowFile);
}
};
}
}

View File

@ -0,0 +1,194 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.util.MockFlowFile;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestPublisherLease {
private ComponentLog logger;
private Producer<byte[], byte[]> producer;
@Before
@SuppressWarnings("unchecked")
public void setup() {
logger = Mockito.mock(ComponentLog.class);
producer = Mockito.mock(Producer.class);
}
@Test
public void testPoisonOnException() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
@Override
public void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = null;
final InputStream failureInputStream = new InputStream() {
@Override
public int read() throws IOException {
throw new IOException("Intentional Unit Test Exception");
}
};
try {
lease.publish(flowFile, failureInputStream, messageKey, demarcatorBytes, topic);
Assert.fail("Expected IOException");
} catch (final IOException ioe) {
// expected
}
assertEquals(1, poisonCount.get());
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
}
@Test
@SuppressWarnings("unchecked")
public void testPoisonOnFailure() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
@Override
public void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = null;
doAnswer(new Answer<Object>() {
@Override
public Object answer(final InvocationOnMock invocation) throws Throwable {
final Callback callback = invocation.getArgumentAt(1, Callback.class);
callback.onCompletion(null, new RuntimeException("Unit Test Intentional Exception"));
return null;
}
}).when(producer).send(any(ProducerRecord.class), any(Callback.class));
lease.publish(flowFile, new ByteArrayInputStream(new byte[1]), messageKey, demarcatorBytes, topic);
assertEquals(1, poisonCount.get());
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
}
@Test
@SuppressWarnings("unchecked")
public void testAllDelimitedMessagesSent() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger) {
@Override
protected void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final AtomicInteger correctMessages = new AtomicInteger(0);
final AtomicInteger incorrectMessages = new AtomicInteger(0);
doAnswer(new Answer<Object>() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
final byte[] value = record.value();
final String valueString = new String(value, StandardCharsets.UTF_8);
if ("1234567890".equals(valueString)) {
correctMessages.incrementAndGet();
} else {
incorrectMessages.incrementAndGet();
}
return null;
}
}).when(producer).send(any(ProducerRecord.class), any(Callback.class));
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = "\n".getBytes(StandardCharsets.UTF_8);
final byte[] flowFileContent = "1234567890\n1234567890\n1234567890\n\n\n\n1234567890\n\n\n1234567890\n\n\n\n".getBytes(StandardCharsets.UTF_8);
lease.publish(flowFile, new ByteArrayInputStream(flowFileContent), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent2 = new byte[0];
lease.publish(new MockFlowFile(2L), new ByteArrayInputStream(flowFileContent2), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent3 = "1234567890\n1234567890".getBytes(StandardCharsets.UTF_8); // no trailing new line
lease.publish(new MockFlowFile(3L), new ByteArrayInputStream(flowFileContent3), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent4 = "\n\n\n".getBytes(StandardCharsets.UTF_8);
lease.publish(new MockFlowFile(4L), new ByteArrayInputStream(flowFileContent4), messageKey, demarcatorBytes, topic);
assertEquals(0, poisonCount.get());
verify(producer, times(0)).flush();
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
assertEquals(7, correctMessages.get());
assertEquals(0, incorrectMessages.get());
verify(producer, times(1)).flush();
}
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.junit.Test;
import org.mockito.Mockito;
public class TestPublisherPool {
@Test
public void testLeaseCloseReturnsToPool() {
final Map<String, Object> kafkaProperties = new HashMap<>();
kafkaProperties.put("bootstrap.servers", "localhost:1111");
kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L);
assertEquals(0, pool.available());
final PublisherLease lease = pool.obtainPublisher();
assertEquals(0, pool.available());
lease.close();
assertEquals(1, pool.available());
}
@Test
public void testPoisonedLeaseNotReturnedToPool() {
final Map<String, Object> kafkaProperties = new HashMap<>();
kafkaProperties.put("bootstrap.servers", "localhost:1111");
kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L);
assertEquals(0, pool.available());
final PublisherLease lease = pool.obtainPublisher();
assertEquals(0, pool.available());
lease.poison();
lease.close();
assertEquals(0, pool.available());
}
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.test;
import java.io.Closeable;
import java.io.IOException;
import java.util.Properties;
import kafka.producer.KeyedMessage;
import kafka.producer.OldProducer;
/**
* Helper class which helps to produce events targeting {@link EmbeddedKafka}
* server.
*/
public class EmbeddedKafkaProducerHelper implements Closeable {
private final EmbeddedKafka kafkaServer;
private final OldProducer producer;
/**
* Will create an instance of EmbeddedKafkaProducerHelper based on default
* configurations.<br>
* Default configuration includes:<br>
* <i>
* metadata.broker.list=[determined from the instance of EmbeddedKafka]<br>
* serializer.class=kafka.serializer.DefaultEncoder<br>
* key.serializer.class=kafka.serializer.DefaultEncoder<br>
* auto.create.topics.enable=true
* </i><br>
* <br>
* If you wish to supply additional configuration properties or override
* existing use
* {@link EmbeddedKafkaProducerHelper#EmbeddedKafkaProducerHelper(EmbeddedKafka, Properties)}
* constructor.
*
* @param kafkaServer
* instance of {@link EmbeddedKafka}
*/
public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer) {
this(kafkaServer, null);
}
/**
* Will create an instance of EmbeddedKafkaProducerHelper based on default
* configurations and additional configuration properties.<br>
* Default configuration includes:<br>
* metadata.broker.list=[determined from the instance of EmbeddedKafka]<br>
* serializer.class=kafka.serializer.DefaultEncoder<br>
* key.serializer.class=kafka.serializer.DefaultEncoder<br>
* auto.create.topics.enable=true<br>
* <br>
*
* @param kafkaServer
* instance of {@link EmbeddedKafka}
* @param additionalProperties
* instance of {@link Properties} specifying additional producer
* configuration properties.
*/
public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer, Properties additionalProperties) {
this.kafkaServer = kafkaServer;
Properties producerProperties = new Properties();
producerProperties.put("metadata.broker.list", "localhost:" + this.kafkaServer.getKafkaPort());
producerProperties.put("serializer.class", "kafka.serializer.DefaultEncoder");
producerProperties.put("key.serializer.class", "kafka.serializer.DefaultEncoder");
producerProperties.put("auto.create.topics.enable", "true");
if (additionalProperties != null) {
producerProperties.putAll(additionalProperties);
}
this.producer = new OldProducer(producerProperties);
}
/**
* Will send an event to a Kafka topic. If topic doesn't exist it will be
* auto-created.
*
* @param topicName
* Kafka topic name.
* @param event
* string representing an event(message) to be sent to Kafka.
*/
public void sendEvent(String topicName, String event) {
KeyedMessage<byte[], byte[]> data = new KeyedMessage<byte[], byte[]>(topicName, event.getBytes());
this.producer.send(data.topic(), data.key(), data.message());
}
/**
* Will close the underlying Kafka producer.
*/
@Override
public void close() throws IOException {
this.producer.close();
}
}

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.kafka;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
@ -111,8 +112,9 @@ class KafkaPublisher implements Closeable {
* instance of {@link PublishingContext} which hold context
* information about the message(s) to be sent.
* @return The index of the last successful offset.
* @throws IOException if unable to read from the Input Stream
*/
KafkaPublisherResult publish(PublishingContext publishingContext) {
KafkaPublisherResult publish(PublishingContext publishingContext) throws IOException {
StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());

View File

@ -223,7 +223,8 @@ public class ConsumeKafka extends AbstractProcessor {
final byte[] demarcator = context.getProperty(ConsumeKafka.MESSAGE_DEMARCATOR).isSet()
? context.getProperty(ConsumeKafka.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
: null;
final Map<String, String> props = new HashMap<>();
final Map<String, Object> props = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());

View File

@ -74,7 +74,7 @@ public class ConsumerPool implements Closeable {
public ConsumerPool(
final int maxConcurrentLeases,
final byte[] demarcator,
final Map<String, String> kafkaProperties,
final Map<String, Object> kafkaProperties,
final List<String> topics,
final long maxWaitMillis,
final String keyEncoding,
@ -115,6 +115,7 @@ public class ConsumerPool implements Closeable {
* sitting idle which could prompt excessive rebalances.
*/
lease = new SimpleConsumerLease(consumer);
/**
* This subscription tightly couples the lease to the given
* consumer. They cannot be separated from then on.
@ -148,7 +149,7 @@ public class ConsumerPool implements Closeable {
});
}
private void closeConsumer(final Consumer consumer) {
private void closeConsumer(final Consumer<?, ?> consumer) {
consumerClosedCountRef.incrementAndGet();
try {
consumer.unsubscribe();

View File

@ -0,0 +1,178 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.nifi.flowfile.FlowFile;
public class InFlightMessageTracker {
private final ConcurrentMap<FlowFile, Counts> messageCountsByFlowFile = new ConcurrentHashMap<>();
private final ConcurrentMap<FlowFile, Exception> failures = new ConcurrentHashMap<>();
private final Object progressMutex = new Object();
public void incrementAcknowledgedCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
counter.incrementAcknowledgedCount();
synchronized (progressMutex) {
progressMutex.notify();
}
}
public int getAcknowledgedCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.get(flowFile);
return (counter == null) ? 0 : counter.getAcknowledgedCount();
}
public void incrementSentCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.computeIfAbsent(flowFile, ff -> new Counts());
counter.incrementSentCount();
}
public int getSentCount(final FlowFile flowFile) {
final Counts counter = messageCountsByFlowFile.get(flowFile);
return (counter == null) ? 0 : counter.getSentCount();
}
public void fail(final FlowFile flowFile, final Exception exception) {
failures.putIfAbsent(flowFile, exception);
synchronized (progressMutex) {
progressMutex.notify();
}
}
public Exception getFailure(final FlowFile flowFile) {
return failures.get(flowFile);
}
public boolean isFailed(final FlowFile flowFile) {
return getFailure(flowFile) != null;
}
public void reset() {
messageCountsByFlowFile.clear();
failures.clear();
}
public PublishResult failOutstanding(final Exception exception) {
messageCountsByFlowFile.keySet().stream()
.filter(ff -> !isComplete(ff))
.filter(ff -> !failures.containsKey(ff))
.forEach(ff -> failures.put(ff, exception));
return createPublishResult();
}
private boolean isComplete(final FlowFile flowFile) {
final Counts counts = messageCountsByFlowFile.get(flowFile);
if (counts.getAcknowledgedCount() == counts.getSentCount()) {
// all messages received successfully.
return true;
}
if (failures.containsKey(flowFile)) {
// FlowFile failed so is complete
return true;
}
return false;
}
private boolean isComplete() {
return messageCountsByFlowFile.keySet().stream()
.allMatch(flowFile -> isComplete(flowFile));
}
void awaitCompletion(final long millis) throws InterruptedException, TimeoutException {
final long startTime = System.nanoTime();
final long maxTime = startTime + TimeUnit.MILLISECONDS.toNanos(millis);
while (System.nanoTime() < maxTime) {
synchronized (progressMutex) {
if (isComplete()) {
return;
}
progressMutex.wait(millis);
}
}
throw new TimeoutException();
}
PublishResult createPublishResult() {
return new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
if (failures.isEmpty()) {
return messageCountsByFlowFile.keySet();
}
final Set<FlowFile> flowFiles = new HashSet<>(messageCountsByFlowFile.keySet());
flowFiles.removeAll(failures.keySet());
return flowFiles;
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return failures.keySet();
}
@Override
public int getSuccessfulMessageCount(final FlowFile flowFile) {
return getAcknowledgedCount(flowFile);
}
@Override
public Exception getReasonForFailure(final FlowFile flowFile) {
return getFailure(flowFile);
}
};
}
public static class Counts {
private final AtomicInteger sentCount = new AtomicInteger(0);
private final AtomicInteger acknowledgedCount = new AtomicInteger(0);
public void incrementSentCount() {
sentCount.incrementAndGet();
}
public void incrementAcknowledgedCount() {
acknowledgedCount.incrementAndGet();
}
public int getAcknowledgedCount() {
return acknowledgedCount.get();
}
public int getSentCount() {
return sentCount.get();
}
}
}

View File

@ -27,8 +27,9 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.SaslConfigs;
@ -186,7 +187,7 @@ final class KafkaProcessorUtils {
final Class<?> classType;
public KafkaConfigValidator(final Class classType) {
public KafkaConfigValidator(final Class<?> classType) {
this.classType = classType;
}
@ -211,7 +212,8 @@ final class KafkaProcessorUtils {
return builder.toString();
}
static void buildCommonKafkaProperties(final ProcessContext context, final Class kafkaConfigClass, final Map<String, String> mapToPopulate) {
static void buildCommonKafkaProperties(final ProcessContext context, final Class<?> kafkaConfigClass, final Map<String, Object> mapToPopulate) {
for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
if (propertyDescriptor.equals(SSL_CONTEXT_SERVICE)) {
// Translate SSLContext Service configuration into Kafka properties
@ -230,28 +232,33 @@ final class KafkaProcessorUtils {
mapToPopulate.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, sslContextService.getTrustStoreType());
}
}
String pName = propertyDescriptor.getName();
String pValue = propertyDescriptor.isExpressionLanguageSupported()
String propertyName = propertyDescriptor.getName();
String propertyValue = propertyDescriptor.isExpressionLanguageSupported()
? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue()
: context.getProperty(propertyDescriptor).getValue();
if (pValue != null) {
if (pName.endsWith(".ms")) { // kafka standard time notation
pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS));
if (propertyValue != null) {
// If the property name ends in ".ms" then it is a time period. We want to accept either an integer as number of milliseconds
// or the standard NiFi time period such as "5 secs"
if (propertyName.endsWith(".ms") && !StringUtils.isNumeric(propertyValue.trim())) { // kafka standard time notation
propertyValue = String.valueOf(FormatUtils.getTimeDuration(propertyValue.trim(), TimeUnit.MILLISECONDS));
}
if (isStaticStringFieldNamePresent(pName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(pName, pValue);
if (isStaticStringFieldNamePresent(propertyName, kafkaConfigClass, CommonClientConfigs.class, SslConfigs.class, SaslConfigs.class)) {
mapToPopulate.put(propertyName, propertyValue);
}
}
}
}
private static boolean isStaticStringFieldNamePresent(final String name, final Class... classes) {
private static boolean isStaticStringFieldNamePresent(final String name, final Class<?>... classes) {
return KafkaProcessorUtils.getPublicStaticStringFieldValues(classes).contains(name);
}
private static Set<String> getPublicStaticStringFieldValues(final Class... classes) {
private static Set<String> getPublicStaticStringFieldValues(final Class<?>... classes) {
final Set<String> strings = new HashSet<>();
for (final Class classType : classes) {
for (final Class<?> classType : classes) {
for (final Field field : classType.getDeclaredFields()) {
if (Modifier.isPublic(field.getModifiers()) && Modifier.isStatic(field.getModifiers()) && field.getType().equals(String.class)) {
try {

View File

@ -1,236 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.stream.io.util.StreamDemarcator;
/**
* Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
* with sending contents of the {@link FlowFile}s to Kafka.
*/
class KafkaPublisher implements Closeable {
private final Producer<byte[], byte[]> kafkaProducer;
private volatile long ackWaitTime = 30000;
private final ComponentLog componentLog;
private final int ackCheckSize;
KafkaPublisher(Properties kafkaProperties, ComponentLog componentLog) {
this(kafkaProperties, 100, componentLog);
}
/**
* Creates an instance of this class as well as the instance of the
* corresponding Kafka {@link KafkaProducer} using provided Kafka
* configuration properties.
*
* @param kafkaProperties instance of {@link Properties} used to bootstrap
* {@link KafkaProducer}
*/
KafkaPublisher(Properties kafkaProperties, int ackCheckSize, ComponentLog componentLog) {
this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
this.ackCheckSize = ackCheckSize;
this.componentLog = componentLog;
}
/**
* Publishes messages to Kafka topic. It uses {@link StreamDemarcator} to
* determine how many messages to Kafka will be sent from a provided
* {@link InputStream} (see {@link PublishingContext#getContentStream()}).
* It supports two publishing modes:
* <ul>
* <li>Sending all messages constructed from
* {@link StreamDemarcator#nextToken()} operation.</li>
* <li>Sending only unacknowledged messages constructed from
* {@link StreamDemarcator#nextToken()} operation.</li>
* </ul>
* The unacknowledged messages are determined from the value of
* {@link PublishingContext#getLastAckedMessageIndex()}.
* <br>
* This method assumes content stream affinity where it is expected that the
* content stream that represents the same Kafka message(s) will remain the
* same across possible retries. This is required specifically for cases
* where delimiter is used and a single content stream may represent
* multiple Kafka messages. The
* {@link PublishingContext#getLastAckedMessageIndex()} will provide the
* index of the last ACKed message, so upon retry only messages with the
* higher index are sent.
*
* @param publishingContext instance of {@link PublishingContext} which hold
* context information about the message(s) to be sent.
* @return The index of the last successful offset.
*/
KafkaPublisherResult publish(PublishingContext publishingContext) {
StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(),
publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize());
int prevLastAckedMessageIndex = publishingContext.getLastAckedMessageIndex();
List<Future<RecordMetadata>> resultFutures = new ArrayList<>();
byte[] messageBytes;
int tokenCounter = 0;
boolean continueSending = true;
KafkaPublisherResult result = null;
for (; continueSending && (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) {
if (prevLastAckedMessageIndex < tokenCounter) {
ProducerRecord<byte[], byte[]> message = new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getKeyBytes(), messageBytes);
resultFutures.add(this.kafkaProducer.send(message));
if (tokenCounter % this.ackCheckSize == 0) {
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
resultFutures.clear();
if (lastAckedMessageIndex % this.ackCheckSize != 0) {
continueSending = false;
result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
}
prevLastAckedMessageIndex = lastAckedMessageIndex;
}
}
}
if (result == null) {
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
resultFutures.clear();
result = new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
}
return result;
}
/**
* Sets the time this publisher will wait for the {@link Future#get()}
* operation (the Future returned by
* {@link KafkaProducer#send(ProducerRecord)}) to complete before timing
* out.
*
* This value will also be used as a timeout when closing the underlying
* {@link KafkaProducer}. See {@link #close()}.
*/
void setAckWaitTime(long ackWaitTime) {
this.ackWaitTime = ackWaitTime;
}
/**
* This operation will process ACKs from Kafka in the order in which
* {@link KafkaProducer#send(ProducerRecord)} invocation were made returning
* the index of the last ACKed message. Within this operation processing ACK
* simply means successful invocation of 'get()' operation on the
* {@link Future} returned by {@link KafkaProducer#send(ProducerRecord)}
* operation. Upon encountering any type of error while interrogating such
* {@link Future} the ACK loop will end. Messages that were not ACKed would
* be considered non-delivered and therefore could be resent at the later
* time.
*
* @param sendFutures list of {@link Future}s representing results of
* publishing to Kafka
*
* @param lastAckMessageIndex the index of the last ACKed message. It is
* important to provide the last ACKed message especially while re-trying so
* the proper index is maintained.
*/
private int processAcks(List<Future<RecordMetadata>> sendFutures, int lastAckMessageIndex) {
boolean exceptionThrown = false;
for (int segmentCounter = 0; segmentCounter < sendFutures.size() && !exceptionThrown; segmentCounter++) {
Future<RecordMetadata> future = sendFutures.get(segmentCounter);
try {
future.get(this.ackWaitTime, TimeUnit.MILLISECONDS);
lastAckMessageIndex++;
} catch (InterruptedException e) {
exceptionThrown = true;
Thread.currentThread().interrupt();
this.warnOrError("Interrupted while waiting for acks from Kafka", null);
} catch (ExecutionException e) {
exceptionThrown = true;
this.warnOrError("Failed while waiting for acks from Kafka", e);
} catch (TimeoutException e) {
exceptionThrown = true;
this.warnOrError("Timed out while waiting for acks from Kafka", null);
}
}
return lastAckMessageIndex;
}
/**
* Will close the underlying {@link KafkaProducer} waiting if necessary for
* the same duration as supplied {@link #setAckWaitTime(long)}
*/
@Override
public void close() {
this.kafkaProducer.close(this.ackWaitTime, TimeUnit.MILLISECONDS);
}
/**
*
*/
private void warnOrError(String message, Exception e) {
if (e == null) {
this.componentLog.warn(message);
} else {
this.componentLog.error(message, e);
}
}
/**
* Encapsulates the result received from publishing messages to Kafka
*/
static class KafkaPublisherResult {
private final int messagesSent;
private final int lastMessageAcked;
KafkaPublisherResult(int messagesSent, int lastMessageAcked) {
this.messagesSent = messagesSent;
this.lastMessageAcked = lastMessageAcked;
}
public int getMessagesSent() {
return this.messagesSent;
}
public int getLastMessageAcked() {
return this.lastMessageAcked;
}
public boolean isAllAcked() {
return this.lastMessageAcked > -1 && this.messagesSent - 1 == this.lastMessageAcked;
}
@Override
public String toString() {
return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
}
}
}

View File

@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
@ -27,17 +28,16 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import javax.xml.bind.DatatypeConverter;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnStopped;
@ -46,17 +46,15 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.FlowFileFilters;
import org.apache.nifi.processor.util.StandardValidators;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Tags({"Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.9.x"})
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka using the Kafka 0.9 producer. "
@ -70,20 +68,10 @@ import org.slf4j.LoggerFactory;
description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+ " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ")
public class PublishKafka extends AbstractSessionFactoryProcessor {
private final Logger logger = LoggerFactory.getLogger(this.getClass());
protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id";
protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx";
protected static final String FAILED_TOPIC_ATTR = "failed.topic";
protected static final String FAILED_KEY_ATTR = "failed.key";
protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter";
@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
+ "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+ "be greater than 1.")
public class PublishKafka extends AbstractProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -127,15 +115,26 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
.defaultValue(DELIVERY_BEST_EFFORT.getValue())
.build();
static final PropertyDescriptor META_WAIT_TIME = new PropertyDescriptor.Builder()
static final PropertyDescriptor METADATA_WAIT_TIME = new PropertyDescriptor.Builder()
.name(ProducerConfig.MAX_BLOCK_MS_CONFIG)
.displayName("Meta Data Wait Time")
.description("The amount of time KafkaConsumer will wait to obtain metadata during the 'send' call before failing the "
.displayName("Max Metadata Wait Time")
.description("The amount of time publisher will wait to obtain metadata or wait for the buffer to flush during the 'send' call before failing the "
+ "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.defaultValue("30 sec")
.defaultValue("5 sec")
.build();
static final PropertyDescriptor ACK_WAIT_TIME = new PropertyDescriptor.Builder()
.name("ack.wait.time")
.displayName("Acknowledgment Wait Time")
.description("After sending a message to Kafka, this indicates the amount of time that we are willing to wait for a response from Kafka. "
+ "If Kafka does not acknowledge the message within this time period, the FlowFile will be routed to 'failure'.")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(false)
.required(true)
.defaultValue("5 secs")
.build();
static final PropertyDescriptor MAX_REQUEST_SIZE = new PropertyDescriptor.Builder()
@ -176,7 +175,7 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
.description("Specifies the string (interpreted as UTF-8) to use for demarcating 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. "
+ "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on your OS.")
+ "To enter special character such as 'new line' use CTRL+Enter or Shift+Enter, depending on your OS.")
.build();
static final PropertyDescriptor PARTITION_CLASS = new PropertyDescriptor.Builder()
@ -208,39 +207,31 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
.build();
static final List<PropertyDescriptor> DESCRIPTORS;
private static final List<PropertyDescriptor> PROPERTIES;
private static final Set<Relationship> RELATIONSHIPS;
static final Set<Relationship> RELATIONSHIPS;
private volatile PublisherPool publisherPool = null;
private volatile String brokers;
private final AtomicInteger taskCounter = new AtomicInteger();
private volatile boolean acceptTask = true;
/*
* Will ensure that list of PropertyDescriptors is build only once, since
* all other lifecycle methods are invoked multiple times.
*/
static {
final List<PropertyDescriptor> _descriptors = new ArrayList<>();
_descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
_descriptors.add(TOPIC);
_descriptors.add(DELIVERY_GUARANTEE);
_descriptors.add(KEY);
_descriptors.add(KEY_ATTRIBUTE_ENCODING);
_descriptors.add(MESSAGE_DEMARCATOR);
_descriptors.add(MAX_REQUEST_SIZE);
_descriptors.add(META_WAIT_TIME);
_descriptors.add(PARTITION_CLASS);
_descriptors.add(COMPRESSION_CODEC);
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
properties.add(TOPIC);
properties.add(DELIVERY_GUARANTEE);
properties.add(KEY);
properties.add(KEY_ATTRIBUTE_ENCODING);
properties.add(MESSAGE_DEMARCATOR);
properties.add(MAX_REQUEST_SIZE);
properties.add(ACK_WAIT_TIME);
properties.add(METADATA_WAIT_TIME);
properties.add(PARTITION_CLASS);
properties.add(COMPRESSION_CODEC);
DESCRIPTORS = Collections.unmodifiableList(_descriptors);
PROPERTIES = Collections.unmodifiableList(properties);
final Set<Relationship> _relationships = new HashSet<>();
_relationships.add(REL_SUCCESS);
_relationships.add(REL_FAILURE);
RELATIONSHIPS = Collections.unmodifiableSet(_relationships);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
RELATIONSHIPS = Collections.unmodifiableSet(relationships);
}
@Override
@ -250,14 +241,16 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return DESCRIPTORS;
return PROPERTIES;
}
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
.name(propertyDescriptorName).addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true)
.name(propertyDescriptorName)
.addValidator(new KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
.dynamic(true)
.build();
}
@ -266,226 +259,123 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
return KafkaProcessorUtils.validateCommonProperties(validationContext);
}
volatile KafkaPublisher kafkaPublisher;
/**
* This thread-safe operation will delegate to
* {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after first
* checking and creating (if necessary) Kafka resource which could be either
* {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close and
* destroy the underlying Kafka resource upon catching an {@link Exception}
* raised by {@link #rendezvousWithKafka(ProcessContext, ProcessSession)}.
* After Kafka resource is destroyed it will be re-created upon the next
* invocation of this operation essentially providing a self healing
* mechanism to deal with potentially corrupted resource.
* <p>
* Keep in mind that upon catching an exception the state of this processor
* will be set to no longer accept any more tasks, until Kafka resource is
* reset. This means that in a multi-threaded situation currently executing
* tasks will be given a chance to complete while no new tasks will be
* accepted.
*
* @param context context
* @param sessionFactory factory
*/
@Override
public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
if (this.acceptTask) { // acts as a circuit breaker to allow existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks are accepted.
this.taskCounter.incrementAndGet();
final ProcessSession session = sessionFactory.createSession();
try {
/*
* We can't be doing double null check here since as a pattern
* it only works for lazy init but not reset, which is what we
* are doing here. In fact the first null check is dangerous
* since 'kafkaPublisher' can become null right after its null
* check passed causing subsequent NPE.
*/
synchronized (this) {
if (this.kafkaPublisher == null) {
this.kafkaPublisher = this.buildKafkaResource(context, session);
}
private synchronized PublisherPool getPublisherPool(final ProcessContext context) {
PublisherPool pool = publisherPool;
if (pool != null) {
return pool;
}
/*
* The 'processed' boolean flag does not imply any failure or success. It simply states that:
* - ConsumeKafka - some messages were received form Kafka and 1_ FlowFile were generated
* - PublishKafka0_10 - some messages were sent to Kafka based on existence of the input FlowFile
*/
boolean processed = this.rendezvousWithKafka(context, session);
session.commit();
if (!processed) {
context.yield();
}
} catch (Throwable e) {
this.acceptTask = false;
session.rollback(true);
this.getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, e});
} finally {
synchronized (this) {
if (this.taskCounter.decrementAndGet() == 0 && !this.acceptTask) {
this.close();
this.acceptTask = true;
}
}
}
} else {
this.logger.debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
this.getLogger().debug("Task was not accepted due to the processor being in 'reset' state. It will be re-submitted upon completion of the reset.");
context.yield();
}
return publisherPool = createPublisherPool(context);
}
protected PublisherPool createPublisherPool(final ProcessContext context) {
final int maxMessageSize = context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue();
final long maxAckWaitMillis = context.getProperty(ACK_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS).longValue();
final Map<String, Object> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put("max.request.size", String.valueOf(maxMessageSize));
return new PublisherPool(kafkaProperties, getLogger(), maxMessageSize, maxAckWaitMillis);
}
/**
* Will call {@link Closeable#close()} on the target resource after which
* the target resource will be set to null. Should only be called when there
* are no more threads being executed on this processor or when it has been
* verified that only a single thread remains.
*
* @see KafkaPublisher
* @see KafkaConsumer
*/
@OnStopped
public void close() {
try {
if (this.kafkaPublisher != null) {
try {
this.kafkaPublisher.close();
} catch (Exception e) {
this.getLogger().warn("Failed while closing " + this.kafkaPublisher, e);
}
}
} finally {
this.kafkaPublisher = null;
}
public void closePool() {
if (publisherPool != null) {
publisherPool.close();
}
/**
* Will rendezvous with Kafka if {@link ProcessSession} contains
* {@link FlowFile} producing a result {@link FlowFile}.
* <br>
* The result {@link FlowFile} that is successful is then transfered to
* {@link #REL_SUCCESS}
* <br>
* The result {@link FlowFile} that is failed is then transfered to
* {@link #REL_FAILURE}
*
*/
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) {
FlowFile flowFile = session.get();
if (flowFile != null) {
long start = System.nanoTime();
flowFile = this.doRendezvousWithKafka(flowFile, context, session);
Relationship relationship = REL_SUCCESS;
if (!this.isFailedFlowFile(flowFile)) {
String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
String transitUri = KafkaProcessorUtils.buildTransitURI(context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(), this.brokers, topic);
session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration);
this.getLogger().debug("Successfully sent {} to Kafka as {} message(s) in {} millis",
new Object[]{flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration});
publisherPool = null;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet();
final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500));
if (flowFiles.isEmpty()) {
return;
}
final PublisherPool pool = getPublisherPool(context);
if (pool == null) {
context.yield();
return;
}
final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue();
final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final long startTime = System.nanoTime();
try (final PublisherLease lease = pool.obtainPublisher()) {
// Send each FlowFile to Kafka asynchronously.
for (final FlowFile flowFile : flowFiles) {
if (!isScheduled()) {
// If stopped, re-queue FlowFile instead of sending it
session.transfer(flowFile);
continue;
}
final byte[] messageKey = getMessageKey(flowFile, context);
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
final byte[] demarcatorBytes;
if (useDemarcator) {
demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8);
} else {
relationship = REL_FAILURE;
flowFile = session.penalize(flowFile);
}
session.transfer(flowFile, relationship);
}
return flowFile != null;
demarcatorBytes = null;
}
/**
* Builds and instance of {@link KafkaPublisher}.
*/
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) {
final Map<String, String> kafkaProps = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProps);
kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProps.put("max.request.size", String.valueOf(context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue()));
this.brokers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
final Properties props = new Properties();
props.putAll(kafkaProps);
KafkaPublisher publisher = new KafkaPublisher(props, this.getLogger());
return publisher;
}
/**
* Will rendezvous with {@link KafkaPublisher} after building
* {@link PublishingContext} and will produce the resulting
* {@link FlowFile}. The resulting FlowFile contains all required
* information to determine if message publishing originated from the
* provided FlowFile has actually succeeded fully, partially or failed
* completely (see {@link #isFailedFlowFile(FlowFile)}.
*/
private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
final AtomicReference<KafkaPublisher.KafkaPublisherResult> publishResultRef = new AtomicReference<>();
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(InputStream contentStream) throws IOException {
PublishingContext publishingContext = PublishKafka.this.buildPublishingContext(flowFile, context, contentStream);
KafkaPublisher.KafkaPublisherResult result = PublishKafka.this.kafkaPublisher.publish(publishingContext);
publishResultRef.set(result);
public void process(final InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
lease.publish(flowFile, in, messageKey, demarcatorBytes, topic);
}
}
});
FlowFile resultFile = publishResultRef.get().isAllAcked()
? this.cleanUpFlowFileIfNecessary(flowFile, session)
: session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context));
if (!this.isFailedFlowFile(resultFile)) {
resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent()));
}
return resultFile;
}
/**
* Builds {@link PublishingContext} for message(s) to be sent to Kafka.
* {@link PublishingContext} contains all contextual information required by
* {@link KafkaPublisher} to publish to Kafka. Such information contains
* things like topic name, content stream, delimiter, key and last ACKed
* message for cases where provided FlowFile is being retried (failed in the
* past).
* <br>
* For the clean FlowFile (file that has been sent for the first time),
* PublishingContext will be built form {@link ProcessContext} associated
* with this invocation.
* <br>
* For the failed FlowFile, {@link PublishingContext} will be built from
* attributes of that FlowFile which by then will already contain required
* information (e.g., topic, key, delimiter etc.). This is required to
* ensure the affinity of the retry in the even where processor
* configuration has changed. However keep in mind that failed FlowFile is
* only considered a failed FlowFile if it is being re-processed by the same
* processor (determined via {@link #FAILED_PROC_ID_ATTR}, see
* {@link #isFailedFlowFile(FlowFile)}). If failed FlowFile is being sent to
* another PublishKafka0_10 processor it is treated as a fresh FlowFile
* regardless if it has #FAILED* attributes set.
*/
private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) {
final byte[] keyBytes = getMessageKey(flowFile, context);
// Complete the send
final PublishResult publishResult = lease.complete();
final String topicName;
final byte[] delimiterBytes;
int lastAckedMessageIndex = -1;
if (this.isFailedFlowFile(flowFile)) {
lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
// Transfer any successful FlowFiles.
final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
for (FlowFile success : publishResult.getSuccessfulFlowFiles()) {
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(success).getValue();
final int msgCount = publishResult.getSuccessfulMessageCount(success);
success = session.putAttribute(success, MSG_COUNT, String.valueOf(msgCount));
session.adjustCounter("Messages Sent", msgCount, true);
final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topic);
session.getProvenanceReporter().send(success, transitUri, "Sent " + msgCount + " messages", transmissionMillis);
session.transfer(success, REL_SUCCESS);
}
// Transfer any failures.
for (final FlowFile failure : publishResult.getFailedFlowFiles()) {
final int successCount = publishResult.getSuccessfulMessageCount(failure);
if (successCount > 0) {
getLogger().error("Failed to send some messages for {} to Kafka, but {} messages were acknowledged by Kafka. Routing to failure due to {}",
new Object[] {failure, successCount, publishResult.getReasonForFailure(failure)});
} else {
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR)
.evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
getLogger().error("Failed to send all message for {} to Kafka; routing to failure due to {}",
new Object[] {failure, publishResult.getReasonForFailure(failure)});
}
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex,
context.getProperty(MAX_REQUEST_SIZE).asDataSize(DataUnit.B).intValue());
publishingContext.setKeyBytes(keyBytes);
publishingContext.setDelimiterBytes(delimiterBytes);
return publishingContext;
session.transfer(failure, REL_FAILURE);
}
}
}
private byte[] getMessageKey(final FlowFile flowFile, final ProcessContext context) {
if (context.getProperty(MESSAGE_DEMARCATOR).isSet()) {
return null;
}
final String uninterpretedKey;
if (context.getProperty(KEY).isSet()) {
uninterpretedKey = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
@ -504,51 +394,4 @@ public class PublishKafka extends AbstractSessionFactoryProcessor {
return DatatypeConverter.parseHexBinary(uninterpretedKey);
}
/**
* Will remove FAILED_* attributes if FlowFile is no longer considered a
* failed FlowFile
*
* @see #isFailedFlowFile(FlowFile)
*/
private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) {
if (this.isFailedFlowFile(flowFile)) {
Set<String> keysToRemove = new HashSet<>();
keysToRemove.add(FAILED_DELIMITER_ATTR);
keysToRemove.add(FAILED_KEY_ATTR);
keysToRemove.add(FAILED_TOPIC_ATTR);
keysToRemove.add(FAILED_PROC_ID_ATTR);
keysToRemove.add(FAILED_LAST_ACK_IDX);
flowFile = session.removeAllAttributes(flowFile, keysToRemove);
}
return flowFile;
}
/**
* Builds a {@link Map} of FAILED_* attributes
*
* @see #FAILED_PROC_ID_ATTR
* @see #FAILED_LAST_ACK_IDX
* @see #FAILED_TOPIC_ATTR
* @see #FAILED_KEY_ATTR
* @see #FAILED_DELIMITER_ATTR
*/
private Map<String, String> buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile, ProcessContext context) {
Map<String, String> attributes = new HashMap<>();
attributes.put(FAILED_PROC_ID_ATTR, this.getIdentifier());
attributes.put(FAILED_LAST_ACK_IDX, String.valueOf(lastAckedMessageIndex));
attributes.put(FAILED_TOPIC_ATTR, context.getProperty(TOPIC).evaluateAttributeExpressions(sourceFlowFile).getValue());
attributes.put(FAILED_KEY_ATTR, context.getProperty(KEY).evaluateAttributeExpressions(sourceFlowFile).getValue());
attributes.put(FAILED_DELIMITER_ATTR, context.getProperty(MESSAGE_DEMARCATOR).isSet()
? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions(sourceFlowFile).getValue() : null);
return attributes;
}
/**
* Returns 'true' if provided FlowFile is a failed FlowFile. A failed
* FlowFile contains {@link #FAILED_PROC_ID_ATTR}.
*/
private boolean isFailedFlowFile(FlowFile flowFile) {
return this.getIdentifier().equals(flowFile.getAttribute(FAILED_PROC_ID_ATTR));
}
}

View File

@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.Collection;
import java.util.Collections;
import org.apache.nifi.flowfile.FlowFile;
public interface PublishResult {
Collection<FlowFile> getSuccessfulFlowFiles();
Collection<FlowFile> getFailedFlowFiles();
int getSuccessfulMessageCount(FlowFile flowFile);
Exception getReasonForFailure(FlowFile flowFile);
public static final PublishResult EMPTY = new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
return Collections.emptyList();
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return Collections.emptyList();
}
@Override
public int getSuccessfulMessageCount(FlowFile flowFile) {
return 0;
}
@Override
public Exception getReasonForFailure(FlowFile flowFile) {
return null;
}
};
}

View File

@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.stream.io.exception.TokenTooLargeException;
import org.apache.nifi.stream.io.util.StreamDemarcator;
public class PublisherLease implements Closeable {
private final ComponentLog logger;
private final Producer<byte[], byte[]> producer;
private final int maxMessageSize;
private final long maxAckWaitMillis;
private volatile boolean poisoned = false;
private InFlightMessageTracker tracker;
public PublisherLease(final Producer<byte[], byte[]> producer, final int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
this.producer = producer;
this.maxMessageSize = maxMessageSize;
this.logger = logger;
this.maxAckWaitMillis = maxAckWaitMillis;
}
protected void poison() {
this.poisoned = true;
}
public boolean isPoisoned() {
return poisoned;
}
void publish(final FlowFile flowFile, final InputStream flowFileContent, final byte[] messageKey, final byte[] demarcatorBytes, final String topic) throws IOException {
if (tracker == null) {
tracker = new InFlightMessageTracker();
}
try (final StreamDemarcator demarcator = new StreamDemarcator(flowFileContent, demarcatorBytes, maxMessageSize)) {
byte[] messageContent;
try {
while ((messageContent = demarcator.nextToken()) != null) {
// We do not want to use any key if we have a demarcator because that would result in
// the key being the same for multiple messages
final byte[] keyToUse = demarcatorBytes == null ? messageKey : null;
publish(flowFile, keyToUse, messageContent, topic, tracker);
if (tracker.isFailed(flowFile)) {
// If we have a failure, don't try to send anything else.
return;
}
}
} catch (final TokenTooLargeException ttle) {
tracker.fail(flowFile, ttle);
}
} catch (final Exception e) {
tracker.fail(flowFile, e);
poison();
throw e;
}
}
private void publish(final FlowFile flowFile, final byte[] messageKey, final byte[] messageContent, final String topic, final InFlightMessageTracker tracker) {
final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, null, messageKey, messageContent);
producer.send(record, new Callback() {
@Override
public void onCompletion(final RecordMetadata metadata, final Exception exception) {
if (exception == null) {
tracker.incrementAcknowledgedCount(flowFile);
} else {
tracker.fail(flowFile, exception);
poison();
}
}
});
tracker.incrementSentCount(flowFile);
}
public PublishResult complete() {
if (tracker == null) {
throw new IllegalStateException("Cannot complete publishing to Kafka because Publisher Lease was already closed");
}
producer.flush();
try {
tracker.awaitCompletion(maxAckWaitMillis);
return tracker.createPublishResult();
} catch (final InterruptedException e) {
logger.warn("Interrupted while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka");
Thread.currentThread().interrupt();
return tracker.failOutstanding(e);
} catch (final TimeoutException e) {
logger.warn("Timed out while waiting for an acknowledgement from Kafka; some FlowFiles may be transferred to 'failure' even though they were received by Kafka");
return tracker.failOutstanding(e);
} finally {
tracker = null;
}
}
@Override
public void close() {
producer.close(maxAckWaitMillis, TimeUnit.MILLISECONDS);
tracker = null;
}
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.Closeable;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.nifi.logging.ComponentLog;
public class PublisherPool implements Closeable {
private final ComponentLog logger;
private final BlockingQueue<PublisherLease> publisherQueue;
private final Map<String, Object> kafkaProperties;
private final int maxMessageSize;
private final long maxAckWaitMillis;
private volatile boolean closed = false;
PublisherPool(final Map<String, Object> kafkaProperties, final ComponentLog logger, final int maxMessageSize, final long maxAckWaitMillis) {
this.logger = logger;
this.publisherQueue = new LinkedBlockingQueue<>();
this.kafkaProperties = kafkaProperties;
this.maxMessageSize = maxMessageSize;
this.maxAckWaitMillis = maxAckWaitMillis;
}
public PublisherLease obtainPublisher() {
if (isClosed()) {
throw new IllegalStateException("Connection Pool is closed");
}
PublisherLease lease = publisherQueue.poll();
if (lease != null) {
return lease;
}
lease = createLease();
return lease;
}
private PublisherLease createLease() {
final Producer<byte[], byte[]> producer = new KafkaProducer<>(kafkaProperties);
final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger) {
@Override
public void close() {
if (isPoisoned() || isClosed()) {
super.close();
} else {
publisherQueue.offer(this);
}
}
};
return lease;
}
public synchronized boolean isClosed() {
return closed;
}
@Override
public synchronized void close() {
closed = true;
PublisherLease lease;
while ((lease = publisherQueue.poll()) != null) {
lease.close();
}
}
/**
* Returns the number of leases that are currently available
*
* @return the number of leases currently available
*/
protected int available() {
return publisherQueue.size();
}
}

View File

@ -1,124 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
/**
* Holder of context information used by {@link KafkaPublisher} required to
* publish messages to Kafka.
*/
class PublishingContext {
private final InputStream contentStream;
private final String topic;
private final int lastAckedMessageIndex;
private final int maxRequestSize;
private byte[] keyBytes;
private byte[] delimiterBytes;
PublishingContext(InputStream contentStream, String topic) {
this(contentStream, topic, -1);
}
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
this(contentStream, topic, lastAckedMessageIndex, 1048576);
}
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex, int maxRequestSize) {
this.validateInput(contentStream, topic, lastAckedMessageIndex);
this.contentStream = contentStream;
this.topic = topic;
this.lastAckedMessageIndex = lastAckedMessageIndex;
this.maxRequestSize = maxRequestSize;
}
@Override
public String toString() {
return "topic: '" + this.topic + "'; delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'";
}
int getLastAckedMessageIndex() {
return this.lastAckedMessageIndex;
}
int getMaxRequestSize() {
return this.maxRequestSize;
}
byte[] getKeyBytes() {
return this.keyBytes;
}
byte[] getDelimiterBytes() {
return this.delimiterBytes;
}
InputStream getContentStream() {
return this.contentStream;
}
String getTopic() {
return this.topic;
}
void setKeyBytes(byte[] keyBytes) {
if (this.keyBytes == null) {
if (keyBytes != null) {
this.assertBytesValid(keyBytes);
this.keyBytes = keyBytes;
}
} else {
throw new IllegalArgumentException("'keyBytes' can only be set once per instance");
}
}
void setDelimiterBytes(byte[] delimiterBytes) {
if (this.delimiterBytes == null) {
if (delimiterBytes != null) {
this.assertBytesValid(delimiterBytes);
this.delimiterBytes = delimiterBytes;
}
} else {
throw new IllegalArgumentException("'delimiterBytes' can only be set once per instance");
}
}
private void assertBytesValid(byte[] bytes) {
if (bytes != null) {
if (bytes.length == 0) {
throw new IllegalArgumentException("'bytes' must not be empty");
}
}
}
private void validateInput(InputStream contentStream, String topic, int lastAckedMessageIndex) {
if (contentStream == null) {
throw new IllegalArgumentException("'contentStream' must not be null");
} else if (topic == null || topic.trim().length() == 0) {
throw new IllegalArgumentException("'topic' must not be null or empty");
} else if (lastAckedMessageIndex < -1) {
throw new IllegalArgumentException("'lastAckedMessageIndex' must be >= -1");
}
}
}

View File

@ -1,306 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult;
import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import kafka.consumer.Consumer;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.ConsumerTimeoutException;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import org.apache.kafka.clients.producer.ProducerConfig;
public class KafkaPublisherTest {
private static EmbeddedKafka kafkaLocal;
private static EmbeddedKafkaProducerHelper producerHelper;
@BeforeClass
public static void beforeClass() {
kafkaLocal = new EmbeddedKafka();
kafkaLocal.start();
producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal);
}
@AfterClass
public static void afterClass() throws Exception {
producerHelper.close();
kafkaLocal.stop();
}
@Test
public void validateSuccessfulSendAsWhole() throws Exception {
InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
String topicName = "validateSuccessfulSendAsWhole";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
KafkaPublisherResult result = publisher.publish(publishingContext);
assertEquals(0, result.getLastMessageAcked());
assertEquals(1, result.getMessagesSent());
contentStream.close();
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
assertNotNull(iter.next());
try {
iter.next();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
}
@Test
public void validateSuccessfulSendAsDelimited() throws Exception {
InputStream contentStream = new ByteArrayInputStream(
"Hello Kafka\nHello Kafka\nHello Kafka\nHello Kafka\n".getBytes(StandardCharsets.UTF_8));
String topicName = "validateSuccessfulSendAsDelimited";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
KafkaPublisherResult result = publisher.publish(publishingContext);
assertEquals(3, result.getLastMessageAcked());
assertEquals(4, result.getMessagesSent());
contentStream.close();
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
assertNotNull(iter.next());
assertNotNull(iter.next());
assertNotNull(iter.next());
assertNotNull(iter.next());
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
}
/*
* This test simulates the condition where not all messages were ACKed by
* Kafka
*/
@Test
public void validateRetries() throws Exception {
byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
InputStream contentStream = new ByteArrayInputStream(testValue);
String topicName = "validateSuccessfulReSendOfFailedSegments";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
// simulates the first re-try
int lastAckedMessageIndex = 1;
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
String m1 = new String(iter.next().message());
String m2 = new String(iter.next().message());
assertEquals("Hello Kafka3", m1);
assertEquals("Hello Kafka4", m2);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
// simulates the second re-try
lastAckedMessageIndex = 2;
contentStream = new ByteArrayInputStream(testValue);
publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
m1 = new String(iter.next().message());
assertEquals("Hello Kafka4", m1);
publisher.close();
}
/*
* Similar to the above test, but it sets the first retry index to the last
* possible message index and second index to an out of bound index. The
* expectation is that no messages will be sent to Kafka
*/
@Test
public void validateRetriesWithWrongIndex() throws Exception {
byte[] testValue = "Hello Kafka1\nHello Kafka2\nHello Kafka3\nHello Kafka4\n".getBytes(StandardCharsets.UTF_8);
InputStream contentStream = new ByteArrayInputStream(testValue);
String topicName = "validateRetriesWithWrongIndex";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
// simulates the first re-try
int lastAckedMessageIndex = 3;
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
// simulates the second re-try
lastAckedMessageIndex = 6;
contentStream = new ByteArrayInputStream(testValue);
publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
publisher.publish(publishingContext);
try {
iter.next();
fail();
} catch (ConsumerTimeoutException e) {
// that's OK since this is the Kafka mechanism to unblock
}
publisher.close();
}
@Test
public void validateWithMultiByteCharactersNoDelimiter() throws Exception {
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
String topicName = "validateWithMultiByteCharacters";
Properties kafkaProperties = this.buildProducerProperties();
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publisher.publish(publishingContext);
publisher.close();
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
String r = new String(iter.next().message(), StandardCharsets.UTF_8);
assertEquals(data, r);
}
@Test
public void validateWithNonDefaultPartitioner() throws Exception {
String data = "fooandbarandbaz";
InputStream contentStream = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
String topicName = "validateWithNonDefaultPartitioner";
Properties kafkaProperties = this.buildProducerProperties();
kafkaProperties.setProperty("partitioner.class", TestPartitioner.class.getName());
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties, mock(ComponentLog.class));
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
publishingContext.setDelimiterBytes("and".getBytes(StandardCharsets.UTF_8));
try {
publisher.publish(publishingContext);
// partitioner should be invoked 3 times
assertTrue(TestPartitioner.counter == 3);
publisher.close();
} finally {
TestPartitioner.counter = 0;
}
}
private Properties buildProducerProperties() {
Properties kafkaProperties = new Properties();
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + kafkaLocal.getKafkaPort());
kafkaProperties.put("auto.create.topics.enable", "true");
return kafkaProperties;
}
private ConsumerIterator<byte[], byte[]> buildConsumer(String topic) {
Properties props = new Properties();
props.put("zookeeper.connect", "localhost:" + kafkaLocal.getZookeeperPort());
props.put("group.id", "test");
props.put("consumer.timeout.ms", "500");
props.put("auto.offset.reset", "smallest");
ConsumerConfig consumerConfig = new ConsumerConfig(props);
ConsumerConnector consumer = Consumer.createJavaConsumerConnector(consumerConfig);
Map<String, Integer> topicCountMap = new HashMap<>(1);
topicCountMap.put(topic, 1);
Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
ConsumerIterator<byte[], byte[]> iter = streams.get(0).iterator();
return iter;
}
public static class TestPartitioner implements Partitioner {
static int counter;
@Override
public void configure(Map<String, ?> configs) {
// nothing to do, test
}
@Override
public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes,
Cluster cluster) {
counter++;
return 0;
}
@Override
public void close() {
counter = 0;
}
}
}

View File

@ -1,375 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.times;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.verify;
public class PublishKafkaTest {
@Test
public void validateCustomSerilaizerDeserializerSettings() throws Exception {
PublishKafka publishKafka = new PublishKafka();
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(PublishKafka.TOPIC, "foo");
runner.setProperty(PublishKafka.META_WAIT_TIME, "3 sec");
runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
runner.assertValid();
runner.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "Foo");
runner.assertNotValid();
}
@Test
public void validatePropertiesValidation() throws Exception {
PublishKafka publishKafka = new PublishKafka();
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "okeydokey:1234");
runner.setProperty(PublishKafka.TOPIC, "foo");
runner.setProperty(PublishKafka.META_WAIT_TIME, "foo");
try {
runner.assertValid();
fail();
} catch (AssertionError e) {
assertTrue(e.getMessage().contains("'max.block.ms' validated against 'foo' is invalid"));
}
}
@Test
public void validateCustomValidation() {
String topicName = "validateCustomValidation";
PublishKafka publishKafka = new PublishKafka();
/*
* Validates that Kerberos principle is required if one of SASL set for
* secirity protocol
*/
TestRunner runner = TestRunners.newTestRunner(publishKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(KafkaProcessorUtils.SECURITY_PROTOCOL, KafkaProcessorUtils.SEC_SASL_PLAINTEXT);
try {
runner.run();
fail();
} catch (Throwable e) {
assertTrue(e.getMessage().contains("'Kerberos Service Name' is invalid because"));
}
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateSingleCharacterDemarcatedMessages() {
String topicName = "validateSingleCharacterDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
runner.enqueue("Hello World\nGoodbye\n1\n2\n3\n4\n5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerA() {
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo");
runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateMultiCharacterDemarcatedMessagesAndCustomPartitionerB() {
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
StubPublishKafka putKafka = new StubPublishKafka(1);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.PARTITION_CLASS, Partitioners.RoundRobinPartitioner.class.getName());
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "foo");
runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(7)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnSendFailureAndThenResendSuccessA() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka.META_WAIT_TIME, "3000 millis");
final String text = "Hello World\nGoodbye\nfail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
putKafka.destroy();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnSendFailureAndThenResendSuccessB() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(1);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
final String text = "Hello World\nGoodbye\nfail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
assertEquals(1, runner.getQueueSize().getObjectCount()); // due to failure
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnFutureGetFailureAndThenResendSuccessFirstMessageFail() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
final String text = "futurefail\nHello World\nGoodbye\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka.REL_FAILURE).get(0);
assertNotNull(ff);
runner.enqueue(ff);
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
// 6 sends due to duplication
verify(producer, times(5)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateOnFutureGetFailureAndThenResendSuccess() throws Exception {
String topicName = "validateSendFailureAndThenResendSuccess";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
runner.setProperty(PublishKafka.META_WAIT_TIME, "500 millis");
final String text = "Hello World\nGoodbye\nfuturefail\n2";
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
MockFlowFile ff = runner.getFlowFilesForRelationship(PublishKafka.REL_FAILURE).get(0);
assertNotNull(ff);
runner.enqueue(ff);
runner.run(1, false);
assertEquals(0, runner.getQueueSize().getObjectCount());
Producer<byte[], byte[]> producer = putKafka.getProducer();
// 6 sends due to duplication
verify(producer, times(6)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateDemarcationIntoEmptyMessages() {
String topicName = "validateDemarcationIntoEmptyMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
final TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(PublishKafka.KEY, "key1");
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "\n");
final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
runner.enqueue(bytes);
runner.run(1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexRightPartialDemarcatedMessages() {
String topicName = "validateComplexRightPartialDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(3)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexLeftPartialDemarcatedMessages() {
String topicName = "validateComplexLeftPartialDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDSTUFF僠>僠I Mean IT!僠<僠WILDSTUFF僠>僠<僠WILDSTUFF僠>僠".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(4)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@SuppressWarnings("unchecked")
@Test
public void validateComplexPartialMatchDemarcatedMessages() {
String topicName = "validateComplexPartialMatchDemarcatedMessages";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.MESSAGE_DEMARCATOR, "僠<僠WILDSTUFF僠>僠");
runner.enqueue("Hello World僠<僠WILDSTUFF僠>僠Goodbye僠<僠WILDBOOMSTUFF僠>僠".getBytes(StandardCharsets.UTF_8));
runner.run(1, false);
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
Producer<byte[], byte[]> producer = putKafka.getProducer();
verify(producer, times(2)).send(Mockito.any(ProducerRecord.class));
runner.shutdown();
}
@Test
public void validateUtf8Key() {
String topicName = "validateUtf8Key";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.KEY, "${myKey}");
final Map<String, String> attributes = Collections.singletonMap("myKey", "key1");
runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1);
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
final Map<Object, Object> msgs = putKafka.getMessagesSent();
assertEquals(1, msgs.size());
final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
assertTrue(Arrays.equals("key1".getBytes(StandardCharsets.UTF_8), msgKey));
}
@Test
public void validateHexKey() {
String topicName = "validateUtf8Key";
StubPublishKafka putKafka = new StubPublishKafka(100);
TestRunner runner = TestRunners.newTestRunner(putKafka);
runner.setProperty(PublishKafka.TOPIC, topicName);
runner.setProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS, "localhost:1234");
runner.setProperty(PublishKafka.KEY_ATTRIBUTE_ENCODING, PublishKafka.HEX_ENCODING);
runner.setProperty(PublishKafka.KEY, "${myKey}");
final Map<String, String> attributes = Collections.singletonMap("myKey", "6B657931");
runner.enqueue("Hello World".getBytes(StandardCharsets.UTF_8), attributes);
runner.run(1);
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
final Map<Object, Object> msgs = putKafka.getMessagesSent();
assertEquals(1, msgs.size());
final byte[] msgKey = (byte[]) msgs.keySet().iterator().next();
assertTrue(Arrays.equals(new byte[] {0x6B, 0x65, 0x79, 0x31}, msgKey));
}
}

View File

@ -1,91 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import org.junit.Test;
public class PublishingContextTest {
@Test
public void failInvalidConstructorArgs() {
try {
new PublishingContext(null, null);
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), null);
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), "");
fail();
} catch (IllegalArgumentException e) {
// success
}
try {
new PublishingContext(mock(InputStream.class), "mytopic", -3);
fail();
} catch (IllegalArgumentException e) {
// success
}
}
@Test
public void validateFullSetting() {
PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic", 3);
publishingContext.setDelimiterBytes("delimiter".getBytes(StandardCharsets.UTF_8));
publishingContext.setKeyBytes("key".getBytes(StandardCharsets.UTF_8));
assertEquals("delimiter", new String(publishingContext.getDelimiterBytes(), StandardCharsets.UTF_8));
assertEquals("key", new String(publishingContext.getKeyBytes(), StandardCharsets.UTF_8));
assertEquals("topic", publishingContext.getTopic());
assertEquals("topic: 'topic'; delimiter: 'delimiter'", publishingContext.toString());
}
@Test
public void validateOnlyOnceSetPerInstance() {
PublishingContext publishingContext = new PublishingContext(mock(InputStream.class), "topic");
publishingContext.setKeyBytes(new byte[]{0});
try {
publishingContext.setKeyBytes(new byte[]{0});
fail();
} catch (IllegalArgumentException e) {
// success
}
publishingContext.setDelimiterBytes(new byte[]{0});
try {
publishingContext.setDelimiterBytes(new byte[]{0});
fail();
} catch (IllegalArgumentException e) {
// success
}
}
}

View File

@ -1,144 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.BOOTSTRAP_SERVERS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class StubPublishKafka extends PublishKafka {
private volatile Producer<byte[], byte[]> producer;
private volatile boolean failed;
private final int ackCheckSize;
private final ExecutorService executor = Executors.newCachedThreadPool();
private final Map<Object, Object> msgsSent = new ConcurrentHashMap<>();
StubPublishKafka(int ackCheckSize) {
this.ackCheckSize = ackCheckSize;
}
public Producer<byte[], byte[]> getProducer() {
return producer;
}
public void destroy() {
this.executor.shutdownNow();
}
public Map<Object, Object> getMessagesSent() {
return new HashMap<>(msgsSent);
}
@SuppressWarnings("unchecked")
@Override
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
throws ProcessException {
final Map<String, String> kafkaProperties = new HashMap<>();
KafkaProcessorUtils.buildCommonKafkaProperties(context, ProducerConfig.class, kafkaProperties);
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
KafkaPublisher publisher;
try {
Field f = PublishKafka.class.getDeclaredField("brokers");
f.setAccessible(true);
f.set(this, context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue());
publisher = (KafkaPublisher) TestUtils.getUnsafe().allocateInstance(KafkaPublisher.class);
publisher.setAckWaitTime(15000);
producer = mock(Producer.class);
this.instrumentProducer(producer, false);
Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer");
kf.setAccessible(true);
kf.set(publisher, producer);
Field componentLogF = KafkaPublisher.class.getDeclaredField("componentLog");
componentLogF.setAccessible(true);
componentLogF.set(publisher, mock(ComponentLog.class));
Field ackCheckSizeField = KafkaPublisher.class.getDeclaredField("ackCheckSize");
ackCheckSizeField.setAccessible(true);
ackCheckSizeField.set(publisher, this.ackCheckSize);
} catch (Exception e) {
e.printStackTrace();
throw new IllegalStateException(e);
}
return publisher;
}
@SuppressWarnings("unchecked")
private void instrumentProducer(Producer<byte[], byte[]> producer, boolean failRandomly) {
when(producer.send(Mockito.any(ProducerRecord.class))).then(new Answer<Future<RecordMetadata>>() {
@Override
public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
if (record != null && record.key() != null) {
msgsSent.put(record.key(), record.value());
}
String value = new String(record.value(), StandardCharsets.UTF_8);
if ("fail".equals(value) && !StubPublishKafka.this.failed) {
StubPublishKafka.this.failed = true;
throw new RuntimeException("intentional");
}
Future<RecordMetadata> future = executor.submit(new Callable<RecordMetadata>() {
@Override
public RecordMetadata call() throws Exception {
if ("futurefail".equals(value) && !StubPublishKafka.this.failed) {
StubPublishKafka.this.failed = true;
throw new TopicAuthorizationException("Unauthorized");
} else {
TopicPartition partition = new TopicPartition("foo", 0);
RecordMetadata meta = new RecordMetadata(partition, 0, 0);
return meta;
}
}
});
return future;
}
});
}
}

View File

@ -0,0 +1,87 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import org.apache.nifi.util.MockFlowFile;
import org.junit.Assert;
import org.junit.Test;
public class TestInFlightMessageTracker {
@Test(timeout = 5000L)
public void testAwaitCompletionWhenComplete() throws InterruptedException, TimeoutException {
final MockFlowFile flowFile = new MockFlowFile(1L);
final InFlightMessageTracker tracker = new InFlightMessageTracker();
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementAcknowledgedCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementAcknowledgedCount(flowFile);
tracker.awaitCompletion(1L);
}
@Test(timeout = 5000L)
public void testAwaitCompletionWhileWaiting() throws InterruptedException, ExecutionException {
final MockFlowFile flowFile = new MockFlowFile(1L);
final InFlightMessageTracker tracker = new InFlightMessageTracker();
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
tracker.incrementSentCount(flowFile);
verifyNotComplete(tracker);
final ExecutorService exec = Executors.newFixedThreadPool(1);
final Future<?> future = exec.submit(() -> {
try {
tracker.awaitCompletion(10000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
tracker.incrementAcknowledgedCount(flowFile);
tracker.incrementAcknowledgedCount(flowFile);
future.get();
}
private void verifyNotComplete(final InFlightMessageTracker tracker) throws InterruptedException {
try {
tracker.awaitCompletion(10L);
Assert.fail("Expected timeout");
} catch (final TimeoutException te) {
// expected
}
}
}

View File

@ -0,0 +1,262 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Before;
import org.junit.Test;
public class TestPublishKafka {
private static final String TOPIC_NAME = "unit-test";
private PublisherPool mockPool;
private PublisherLease mockLease;
private TestRunner runner;
@Before
public void setup() {
mockPool = mock(PublisherPool.class);
mockLease = mock(PublisherLease.class);
when(mockPool.obtainPublisher()).thenReturn(mockLease);
runner = TestRunners.newTestRunner(new PublishKafka() {
@Override
protected PublisherPool createPublisherPool(final ProcessContext context) {
return mockPool;
}
});
runner.setProperty(PublishKafka.TOPIC, TOPIC_NAME);
}
@Test
public void testSingleSuccess() throws IOException {
final MockFlowFile flowFile = runner.enqueue("hello world");
when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFile, 1));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleSuccess() throws IOException {
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFiles, 1));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 3);
verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
}
@Test
public void testSingleFailure() throws IOException {
final MockFlowFile flowFile = runner.enqueue("hello world");
when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka.REL_FAILURE, 1);
verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleFailures() throws IOException {
final Set<FlowFile> flowFiles = new HashSet<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFiles));
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka.REL_FAILURE, 3);
verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
}
@Test
public void testMultipleMessagesPerFlowFile() throws IOException {
final List<FlowFile> flowFiles = new ArrayList<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
msgCounts.put(flowFiles.get(0), 10);
msgCounts.put(flowFiles.get(1), 20);
final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap());
when(mockLease.complete()).thenReturn(result);
runner.run();
runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 2);
verify(mockLease, times(2)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(0)).poison();
verify(mockLease, times(1)).close();
runner.assertAllFlowFilesContainAttribute("msg.count");
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
.filter(ff -> ff.getAttribute("msg.count").equals("10"))
.count());
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
.filter(ff -> ff.getAttribute("msg.count").equals("20"))
.count());
}
@Test
public void testSomeSuccessSomeFailure() throws IOException {
final List<FlowFile> flowFiles = new ArrayList<>();
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
flowFiles.add(runner.enqueue("hello world"));
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
msgCounts.put(flowFiles.get(0), 10);
msgCounts.put(flowFiles.get(1), 20);
final Map<FlowFile, Exception> failureMap = new HashMap<>();
failureMap.put(flowFiles.get(2), new RuntimeException("Intentional Unit Test Exception"));
failureMap.put(flowFiles.get(3), new RuntimeException("Intentional Unit Test Exception"));
final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles.subList(0, 2)), failureMap);
when(mockLease.complete()).thenReturn(result);
runner.run();
runner.assertTransferCount(PublishKafka.REL_SUCCESS, 2);
runner.assertTransferCount(PublishKafka.REL_FAILURE, 2);
verify(mockLease, times(4)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME));
verify(mockLease, times(1)).complete();
verify(mockLease, times(1)).close();
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
.filter(ff -> "10".equals(ff.getAttribute("msg.count")))
.count());
assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka.REL_SUCCESS).stream()
.filter(ff -> "20".equals(ff.getAttribute("msg.count")))
.count());
assertTrue(runner.getFlowFilesForRelationship(PublishKafka.REL_FAILURE).stream()
.noneMatch(ff -> ff.getAttribute("msg.count") != null));
}
private PublishResult createAllSuccessPublishResult(final FlowFile successfulFlowFile, final int msgCount) {
return createAllSuccessPublishResult(Collections.singleton(successfulFlowFile), msgCount);
}
private PublishResult createAllSuccessPublishResult(final Set<FlowFile> successfulFlowFiles, final int msgCountPerFlowFile) {
final Map<FlowFile, Integer> msgCounts = new HashMap<>();
for (final FlowFile ff : successfulFlowFiles) {
msgCounts.put(ff, msgCountPerFlowFile);
}
return createPublishResult(msgCounts, successfulFlowFiles, Collections.emptyMap());
}
private PublishResult createFailurePublishResult(final FlowFile failure) {
return createFailurePublishResult(Collections.singleton(failure));
}
private PublishResult createFailurePublishResult(final Set<FlowFile> failures) {
final Map<FlowFile, Exception> failureMap = failures.stream().collect(Collectors.toMap(ff -> ff, ff -> new RuntimeException("Intentional Unit Test Exception")));
return createPublishResult(Collections.emptyMap(), Collections.emptySet(), failureMap);
}
private PublishResult createPublishResult(final Map<FlowFile, Integer> msgCounts, final Set<FlowFile> successFlowFiles, final Map<FlowFile, Exception> failures) {
// sanity check.
for (final FlowFile success : successFlowFiles) {
if (failures.containsKey(success)) {
throw new IllegalArgumentException("Found same FlowFile in both 'success' and 'failures' collections: " + success);
}
}
return new PublishResult() {
@Override
public Collection<FlowFile> getSuccessfulFlowFiles() {
return successFlowFiles;
}
@Override
public Collection<FlowFile> getFailedFlowFiles() {
return failures.keySet();
}
@Override
public int getSuccessfulMessageCount(FlowFile flowFile) {
Integer count = msgCounts.get(flowFile);
return count == null ? 0 : count.intValue();
}
@Override
public Exception getReasonForFailure(FlowFile flowFile) {
return failures.get(flowFile);
}
};
}
}

View File

@ -0,0 +1,194 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.util.MockFlowFile;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestPublisherLease {
private ComponentLog logger;
private Producer<byte[], byte[]> producer;
@Before
@SuppressWarnings("unchecked")
public void setup() {
logger = Mockito.mock(ComponentLog.class);
producer = Mockito.mock(Producer.class);
}
@Test
public void testPoisonOnException() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
@Override
public void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = null;
final InputStream failureInputStream = new InputStream() {
@Override
public int read() throws IOException {
throw new IOException("Intentional Unit Test Exception");
}
};
try {
lease.publish(flowFile, failureInputStream, messageKey, demarcatorBytes, topic);
Assert.fail("Expected IOException");
} catch (final IOException ioe) {
// expected
}
assertEquals(1, poisonCount.get());
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
}
@Test
@SuppressWarnings("unchecked")
public void testPoisonOnFailure() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) {
@Override
public void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = null;
doAnswer(new Answer<Object>() {
@Override
public Object answer(final InvocationOnMock invocation) throws Throwable {
final Callback callback = invocation.getArgumentAt(1, Callback.class);
callback.onCompletion(null, new RuntimeException("Unit Test Intentional Exception"));
return null;
}
}).when(producer).send(any(ProducerRecord.class), any(Callback.class));
lease.publish(flowFile, new ByteArrayInputStream(new byte[1]), messageKey, demarcatorBytes, topic);
assertEquals(1, poisonCount.get());
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
}
@Test
@SuppressWarnings("unchecked")
public void testAllDelimitedMessagesSent() throws IOException {
final AtomicInteger poisonCount = new AtomicInteger(0);
final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger) {
@Override
protected void poison() {
poisonCount.incrementAndGet();
super.poison();
}
};
final AtomicInteger correctMessages = new AtomicInteger(0);
final AtomicInteger incorrectMessages = new AtomicInteger(0);
doAnswer(new Answer<Object>() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class);
final byte[] value = record.value();
final String valueString = new String(value, StandardCharsets.UTF_8);
if ("1234567890".equals(valueString)) {
correctMessages.incrementAndGet();
} else {
incorrectMessages.incrementAndGet();
}
return null;
}
}).when(producer).send(any(ProducerRecord.class), any(Callback.class));
final FlowFile flowFile = new MockFlowFile(1L);
final String topic = "unit-test";
final byte[] messageKey = null;
final byte[] demarcatorBytes = "\n".getBytes(StandardCharsets.UTF_8);
final byte[] flowFileContent = "1234567890\n1234567890\n1234567890\n\n\n\n1234567890\n\n\n1234567890\n\n\n\n".getBytes(StandardCharsets.UTF_8);
lease.publish(flowFile, new ByteArrayInputStream(flowFileContent), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent2 = new byte[0];
lease.publish(new MockFlowFile(2L), new ByteArrayInputStream(flowFileContent2), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent3 = "1234567890\n1234567890".getBytes(StandardCharsets.UTF_8); // no trailing new line
lease.publish(new MockFlowFile(3L), new ByteArrayInputStream(flowFileContent3), messageKey, demarcatorBytes, topic);
final byte[] flowFileContent4 = "\n\n\n".getBytes(StandardCharsets.UTF_8);
lease.publish(new MockFlowFile(4L), new ByteArrayInputStream(flowFileContent4), messageKey, demarcatorBytes, topic);
assertEquals(0, poisonCount.get());
verify(producer, times(0)).flush();
final PublishResult result = lease.complete();
assertTrue(result.getFailedFlowFiles().contains(flowFile));
assertFalse(result.getSuccessfulFlowFiles().contains(flowFile));
assertEquals(7, correctMessages.get());
assertEquals(0, incorrectMessages.get());
verify(producer, times(1)).flush();
}
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.pubsub;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.Map;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.logging.ComponentLog;
import org.junit.Test;
import org.mockito.Mockito;
public class TestPublisherPool {
@Test
public void testLeaseCloseReturnsToPool() {
final Map<String, Object> kafkaProperties = new HashMap<>();
kafkaProperties.put("bootstrap.servers", "localhost:1111");
kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L);
assertEquals(0, pool.available());
final PublisherLease lease = pool.obtainPublisher();
assertEquals(0, pool.available());
lease.close();
assertEquals(1, pool.available());
}
@Test
public void testPoisonedLeaseNotReturnedToPool() {
final Map<String, Object> kafkaProperties = new HashMap<>();
kafkaProperties.put("bootstrap.servers", "localhost:1111");
kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L);
assertEquals(0, pool.available());
final PublisherLease lease = pool.obtainPublisher();
assertEquals(0, pool.available());
lease.poison();
lease.close();
assertEquals(0, pool.available());
}
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.kafka.test;
import java.io.Closeable;
import java.io.IOException;
import java.util.Properties;
import kafka.producer.KeyedMessage;
import kafka.producer.OldProducer;
/**
* Helper class which helps to produce events targeting {@link EmbeddedKafka}
* server.
*/
public class EmbeddedKafkaProducerHelper implements Closeable {
private final EmbeddedKafka kafkaServer;
private final OldProducer producer;
/**
* Will create an instance of EmbeddedKafkaProducerHelper based on default
* configurations.<br>
* Default configuration includes:<br>
* <i>
* metadata.broker.list=[determined from the instance of EmbeddedKafka]<br>
* serializer.class=kafka.serializer.DefaultEncoder<br>
* key.serializer.class=kafka.serializer.DefaultEncoder<br>
* auto.create.topics.enable=true
* </i><br>
* <br>
* If you wish to supply additional configuration properties or override
* existing use
* {@link EmbeddedKafkaProducerHelper#EmbeddedKafkaProducerHelper(EmbeddedKafka, Properties)}
* constructor.
*
* @param kafkaServer
* instance of {@link EmbeddedKafka}
*/
public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer) {
this(kafkaServer, null);
}
/**
* Will create an instance of EmbeddedKafkaProducerHelper based on default
* configurations and additional configuration properties.<br>
* Default configuration includes:<br>
* metadata.broker.list=[determined from the instance of EmbeddedKafka]<br>
* serializer.class=kafka.serializer.DefaultEncoder<br>
* key.serializer.class=kafka.serializer.DefaultEncoder<br>
* auto.create.topics.enable=true<br>
* <br>
*
* @param kafkaServer
* instance of {@link EmbeddedKafka}
* @param additionalProperties
* instance of {@link Properties} specifying additional producer
* configuration properties.
*/
public EmbeddedKafkaProducerHelper(EmbeddedKafka kafkaServer, Properties additionalProperties) {
this.kafkaServer = kafkaServer;
Properties producerProperties = new Properties();
producerProperties.put("metadata.broker.list", "localhost:" + this.kafkaServer.getKafkaPort());
producerProperties.put("serializer.class", "kafka.serializer.DefaultEncoder");
producerProperties.put("key.serializer.class", "kafka.serializer.DefaultEncoder");
producerProperties.put("auto.create.topics.enable", "true");
if (additionalProperties != null) {
producerProperties.putAll(additionalProperties);
}
this.producer = new OldProducer(producerProperties);
}
/**
* Will send an event to a Kafka topic. If topic doesn't exist it will be
* auto-created.
*
* @param topicName
* Kafka topic name.
* @param event
* string representing an event(message) to be sent to Kafka.
*/
public void sendEvent(String topicName, String event) {
KeyedMessage<byte[], byte[]> data = new KeyedMessage<byte[], byte[]>(topicName, event.getBytes());
this.producer.send(data.topic(), data.key(), data.message());
}
/**
* Will close the underlying Kafka producer.
*/
@Override
public void close() throws IOException {
this.producer.close();
}
}