mirror of https://github.com/apache/nifi.git
NIFI-1296, NIFI-1680, NIFI-1764, NIFI-1837, NIFI-1827, NIFI-1699 implemented new Kafka processors that leverage Kafka 0.9 API
- Improved StreamScanner for better performance - Renamed StreamScanner to StreamDemarcator as suggested by Joe - Added failure handling logic to ensure both processors can be reset to their initial state (as if they were just started) - Provided comprehensive test suite to validate various aspects of both Publish and Consume from Kafka - Added relevant javadocs - Added initial additionalDetails docs - Addressed NPE reported by NIFI-1764 - Life-cycle refactoring for the existing PutKafka to ensure producer restart after errors - Incorporated code changes contributed by Ralph Perko (see NIFI-1837) - Addressed partition issue in RoundRobinPartitioner discussed in NIFI-1827 - Updated PropertyDescriptor descriptions to reflect their purpose NIFI-1296 added @Ignore on some Kafka tests to improve test time NIFI-1296 reworked tests to avoid dependency on embedded Kafka NIFI-1296 fixed spelling error NIFI-1296 fixed trailing whitespaces in non-java files This closes #366
This commit is contained in:
parent
c13fb3159f
commit
2d03489ec5
|
@ -157,6 +157,11 @@ language governing permissions and limitations under the License. -->
|
|||
<artifactId>nifi-kafka-nar</artifactId>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-pubsub-nar</artifactId>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-http-context-map-nar</artifactId>
|
||||
|
|
|
@ -0,0 +1,191 @@
|
|||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* The <code>StreamDemarcator</code> class takes an input stream and demarcates
|
||||
* it so it could be read (see {@link #nextToken()}) as individual byte[]
|
||||
* demarcated by the provided delimiter. If delimiter is not provided the entire
|
||||
* stream will be read into a single token which may result in
|
||||
* {@link OutOfMemoryError} if stream is too large.
|
||||
*/
|
||||
public class StreamDemarcator {
|
||||
|
||||
private final static int INIT_BUFFER_SIZE = 8192;
|
||||
|
||||
private final InputStream is;
|
||||
|
||||
private final byte[] delimiterBytes;
|
||||
|
||||
private final int maxDataSize;
|
||||
|
||||
private final int initialBufferSize;
|
||||
|
||||
|
||||
private byte[] buffer;
|
||||
|
||||
private int index;
|
||||
|
||||
private int mark;
|
||||
|
||||
private int readAheadLength;
|
||||
|
||||
/**
|
||||
* Constructs a new instance
|
||||
*
|
||||
* @param is
|
||||
* instance of {@link InputStream} representing the data
|
||||
* @param delimiterBytes
|
||||
* byte array representing delimiter bytes used to split the
|
||||
* input stream. Can be null
|
||||
* @param maxDataSize
|
||||
* maximum size of data derived from the input stream. This means
|
||||
* that neither {@link InputStream} nor its individual chunks (if
|
||||
* delimiter is used) can ever be greater then this size.
|
||||
*/
|
||||
public StreamDemarcator(InputStream is, byte[] delimiterBytes, int maxDataSize) {
|
||||
this(is, delimiterBytes, maxDataSize, INIT_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new instance
|
||||
*
|
||||
* @param is
|
||||
* instance of {@link InputStream} representing the data
|
||||
* @param delimiterBytes
|
||||
* byte array representing delimiter bytes used to split the
|
||||
* input stream. Can be null
|
||||
* @param maxDataSize
|
||||
* maximum size of data derived from the input stream. This means
|
||||
* that neither {@link InputStream} nor its individual chunks (if
|
||||
* delimiter is used) can ever be greater then this size.
|
||||
* @param initialBufferSize
|
||||
* initial size of the buffer used to buffer {@link InputStream}
|
||||
* or its parts (if delimiter is used) to create its byte[]
|
||||
* representation. Must be positive integer. The buffer will grow
|
||||
* automatically as needed up to the Integer.MAX_VALUE;
|
||||
*
|
||||
*/
|
||||
public StreamDemarcator(InputStream is, byte[] delimiterBytes, int maxDataSize, int initialBufferSize) {
|
||||
this.validateInput(is, delimiterBytes, maxDataSize, initialBufferSize);
|
||||
this.is = is;
|
||||
this.delimiterBytes = delimiterBytes;
|
||||
this.initialBufferSize = initialBufferSize;
|
||||
this.buffer = new byte[initialBufferSize];
|
||||
this.maxDataSize = maxDataSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will read the next data token from the {@link InputStream} returning null
|
||||
* when it reaches the end of the stream.
|
||||
*/
|
||||
public byte[] nextToken() {
|
||||
byte[] data = null;
|
||||
int j = 0;
|
||||
|
||||
while (data == null && this.buffer != null) {
|
||||
if (this.index >= this.readAheadLength) {
|
||||
this.fill();
|
||||
}
|
||||
if (this.index >= this.readAheadLength) {
|
||||
data = this.extractDataToken(0);
|
||||
this.buffer = null;
|
||||
} else {
|
||||
byte byteVal = this.buffer[this.index++];
|
||||
if (this.delimiterBytes != null && this.delimiterBytes[j] == byteVal) {
|
||||
if (++j == this.delimiterBytes.length) {
|
||||
data = this.extractDataToken(this.delimiterBytes.length);
|
||||
this.mark = this.index;
|
||||
j = 0;
|
||||
}
|
||||
} else {
|
||||
j = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will fill the current buffer from current 'index' position, expanding it
|
||||
* and or shuffling it if necessary
|
||||
*/
|
||||
private void fill() {
|
||||
if (this.index >= this.buffer.length) {
|
||||
if (this.mark == 0) { // expand
|
||||
byte[] newBuff = new byte[this.buffer.length + this.initialBufferSize];
|
||||
System.arraycopy(this.buffer, 0, newBuff, 0, this.buffer.length);
|
||||
this.buffer = newBuff;
|
||||
} else { // shuffle
|
||||
int length = this.index - this.mark;
|
||||
System.arraycopy(this.buffer, this.mark, this.buffer, 0, length);
|
||||
this.index = length;
|
||||
this.mark = 0;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
int bytesRead;
|
||||
do {
|
||||
bytesRead = this.is.read(this.buffer, this.index, this.buffer.length - this.index);
|
||||
} while (bytesRead == 0);
|
||||
|
||||
if (bytesRead != -1) {
|
||||
this.readAheadLength = this.index + bytesRead;
|
||||
if (this.readAheadLength > this.maxDataSize) {
|
||||
throw new IllegalStateException("Maximum allowed data size of " + this.maxDataSize + " exceeded.");
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Failed while reading InputStream", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will extract data token from the current buffer. The length of the data
|
||||
* token is between the current 'mark' and 'index' minus 'lengthSubtract'
|
||||
* which signifies the length of the delimiter (if any). If the above
|
||||
* subtraction results in length 0, null is returned.
|
||||
*/
|
||||
private byte[] extractDataToken(int lengthSubtract) {
|
||||
byte[] data = null;
|
||||
int length = this.index - this.mark - lengthSubtract;
|
||||
if (length > 0) {
|
||||
data = new byte[length];
|
||||
System.arraycopy(this.buffer, this.mark, data, 0, data.length);
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private void validateInput(InputStream is, byte[] delimiterBytes, int maxDataSize, int initialBufferSize) {
|
||||
if (is == null) {
|
||||
throw new IllegalArgumentException("'is' must not be null");
|
||||
} else if (maxDataSize <= 0) {
|
||||
throw new IllegalArgumentException("'maxDataSize' must be > 0");
|
||||
} else if (initialBufferSize <= 0) {
|
||||
throw new IllegalArgumentException("'initialBufferSize' must be > 0");
|
||||
} else if (delimiterBytes != null && delimiterBytes.length == 0){
|
||||
throw new IllegalArgumentException("'delimiterBytes' is an optional argument, but when provided its length must be > 0");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,164 +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.stream.io.util;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class StreamScanner {
|
||||
|
||||
private final static int EOF = -1;
|
||||
|
||||
private final InputStream is;
|
||||
|
||||
private final byte[] delimiterBytes;
|
||||
|
||||
private final int maxDataSize;
|
||||
|
||||
private ByteBuffer buffer;
|
||||
|
||||
private byte[] data;
|
||||
|
||||
/**
|
||||
* Constructs a new instance
|
||||
*
|
||||
* @param is
|
||||
* instance of {@link InputStream} representing the data
|
||||
* @param delimiterBytes
|
||||
* byte array representing delimiter bytes used to split the
|
||||
* input stream. Can be null
|
||||
* @param maxDataSize
|
||||
* maximum size of data derived from the input stream. This means
|
||||
* that neither {@link InputStream} nor its individual chunks (if
|
||||
* delimiter is used) can ever be greater then this size.
|
||||
*/
|
||||
public StreamScanner(InputStream is, byte[] delimiterBytes, int maxDataSize) {
|
||||
this(is, delimiterBytes, maxDataSize, 8192);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new instance
|
||||
*
|
||||
* @param is
|
||||
* instance of {@link InputStream} representing the data
|
||||
* @param delimiterBytes
|
||||
* byte array representing delimiter bytes used to split the
|
||||
* input stream. Can be null
|
||||
* @param maxDataSize
|
||||
* maximum size of data derived from the input stream. This means
|
||||
* that neither {@link InputStream} nor its individual chunks (if
|
||||
* delimiter is used) can ever be greater then this size.
|
||||
* @param initialBufferSize
|
||||
* initial size of the buffer used to buffer {@link InputStream}
|
||||
* or its parts (if delimiter is used) to create its byte[]
|
||||
* representation. Must be positive integer. The buffer will grow
|
||||
* automatically as needed up to the Integer.MAX_VALUE;
|
||||
*
|
||||
*/
|
||||
public StreamScanner(InputStream is, byte[] delimiterBytes, int maxDataSize, int initialBufferSize) {
|
||||
this.is = new BufferedInputStream(is);
|
||||
this.delimiterBytes = delimiterBytes;
|
||||
this.buffer = ByteBuffer.allocate(initialBufferSize);
|
||||
this.maxDataSize = maxDataSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if there are more elements in the stream. This operation is
|
||||
* idempotent.
|
||||
*
|
||||
* @return <i>true</i> if there are more elements in the stream or
|
||||
* <i>false</i> when it reaches the end of the stream after the last
|
||||
* element was retrieved via {@link #next()} operation.
|
||||
*/
|
||||
public boolean hasNext() {
|
||||
int j = 0;
|
||||
int readVal = 0;
|
||||
while (this.data == null && readVal != EOF) {
|
||||
this.expandBufferIfNecessary();
|
||||
try {
|
||||
readVal = this.is.read();
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Failed while reading InputStream", e);
|
||||
}
|
||||
if (readVal == EOF) {
|
||||
this.extractDataToken(0);
|
||||
} else {
|
||||
byte byteVal = (byte)readVal;
|
||||
this.buffer.put(byteVal);
|
||||
if (this.buffer.position() > this.maxDataSize) {
|
||||
throw new IllegalStateException("Maximum allowed data size of " + this.maxDataSize + " exceeded.");
|
||||
}
|
||||
if (this.delimiterBytes != null && this.delimiterBytes[j] == byteVal) {
|
||||
if (++j == this.delimiterBytes.length) {
|
||||
this.extractDataToken(this.delimiterBytes.length);
|
||||
j = 0;
|
||||
}
|
||||
} else {
|
||||
j = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
return this.data != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return byte array representing the next segment in the stream or the
|
||||
* whole stream if no delimiter is used
|
||||
*/
|
||||
public byte[] next() {
|
||||
try {
|
||||
return this.data;
|
||||
} finally {
|
||||
this.data = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private void expandBufferIfNecessary() {
|
||||
if (this.buffer.position() == Integer.MAX_VALUE ){
|
||||
throw new IllegalStateException("Internal buffer has reached the capacity and can not be expended any further");
|
||||
}
|
||||
if (this.buffer.remaining() == 0) {
|
||||
this.buffer.flip();
|
||||
int pos = this.buffer.capacity();
|
||||
int newSize = this.buffer.capacity() * 2 > Integer.MAX_VALUE ? Integer.MAX_VALUE : this.buffer.capacity() * 2;
|
||||
ByteBuffer bb = ByteBuffer.allocate(newSize);
|
||||
bb.put(this.buffer);
|
||||
this.buffer = bb;
|
||||
this.buffer.position(pos);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private void extractDataToken(int lengthSubtract) {
|
||||
this.buffer.flip();
|
||||
if (this.buffer.limit() > 0){ // something must be in the buffer; at least delimiter (e.g., \n)
|
||||
this.data = new byte[this.buffer.limit() - lengthSubtract];
|
||||
this.buffer.get(this.data);
|
||||
}
|
||||
this.buffer.clear();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,227 @@
|
|||
/*
|
||||
* 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.util;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
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.Arrays;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class StreamDemarcatorTest {
|
||||
|
||||
@Test
|
||||
public void validateInitializationFailure() {
|
||||
try {
|
||||
new StreamDemarcator(null, null, -1);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
|
||||
try {
|
||||
new StreamDemarcator(mock(InputStream.class), null, -1);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
|
||||
try {
|
||||
new StreamDemarcator(mock(InputStream.class), null, 10, -1);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
|
||||
try {
|
||||
new StreamDemarcator(mock(InputStream.class), new byte[0], 10, 1);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateNoDelimiter() {
|
||||
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);
|
||||
assertTrue(Arrays.equals(data.getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
// validate that subsequent invocations of nextToken() do not result in exception
|
||||
assertNull(scanner.nextToken());
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateNoDelimiterSmallInitialBuffer() {
|
||||
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);
|
||||
assertTrue(Arrays.equals(data.getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateSingleByteDelimiter() {
|
||||
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);
|
||||
assertTrue(Arrays.equals("Learn from yesterday".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" live for today".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" hope for tomorrow. The important thing is not to stop questioning.".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateDelimiterAtTheBeginning() {
|
||||
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);
|
||||
assertTrue(Arrays.equals("Learn from yesterday".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" live for today".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" hope for tomorrow. The important thing is not to stop questioning.".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateEmptyDelimiterSegments() {
|
||||
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);
|
||||
assertTrue(Arrays.equals("Learn from yesterday".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" live for today".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" hope for tomorrow. The important thing is not to stop questioning.".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateSingleByteDelimiterSmallInitialBuffer() {
|
||||
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);
|
||||
assertTrue(Arrays.equals("Learn from yesterday".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" live for today".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals(" hope for tomorrow. The important thing is not to stop questioning.".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteDelimiter() {
|
||||
String data = "foodaabardaabazzz";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
|
||||
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
|
||||
assertTrue(Arrays.equals("foo".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("bar".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("bazzz".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteDelimiterAtTheBeginning() {
|
||||
String data = "daafoodaabardaabazzz";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
|
||||
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000);
|
||||
assertTrue(Arrays.equals("foo".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("bar".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("bazzz".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteDelimiterSmallInitialBuffer() {
|
||||
String data = "foodaabarffdaabazz";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
|
||||
StreamDemarcator scanner = new StreamDemarcator(is, "daa".getBytes(StandardCharsets.UTF_8), 1000, 1);
|
||||
assertTrue(Arrays.equals("foo".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("barff".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertTrue(Arrays.equals("bazz".getBytes(StandardCharsets.UTF_8), scanner.nextToken()));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteCharsNoDelimiter() {
|
||||
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);
|
||||
byte[] next = scanner.nextToken();
|
||||
assertNotNull(next);
|
||||
assertEquals(data, new String(next, StandardCharsets.UTF_8));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteCharsNoDelimiterSmallInitialBuffer() {
|
||||
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);
|
||||
byte[] next = scanner.nextToken();
|
||||
assertNotNull(next);
|
||||
assertEquals(data, new String(next, StandardCharsets.UTF_8));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithComplexDelimiter() {
|
||||
String data = "THIS IS MY TEXT<MYDEIMITER>THIS IS MY NEW TEXT<MYDEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamDemarcator scanner = new StreamDemarcator(is, "<MYDEIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
|
||||
assertEquals("THIS IS MY TEXT", new String(scanner.nextToken(), StandardCharsets.UTF_8));
|
||||
assertEquals("THIS IS MY NEW TEXT", new String(scanner.nextToken(), StandardCharsets.UTF_8));
|
||||
assertEquals("THIS IS MY NEWEST TEXT", new String(scanner.nextToken(), StandardCharsets.UTF_8));
|
||||
assertNull(scanner.nextToken());
|
||||
}
|
||||
|
||||
@Test(expected = IllegalStateException.class)
|
||||
public void validateMaxBufferSize() {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamDemarcator scanner = new StreamDemarcator(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 20);
|
||||
scanner.nextToken();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateScannerHandlesNegativeOneByteInputsNoDelimiter() {
|
||||
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();
|
||||
assertArrayEquals(b, new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateScannerHandlesNegativeOneByteInputs() {
|
||||
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();
|
||||
assertArrayEquals(b, new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyScannerHandlesNegativeOneByteDelimiter() {
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(new byte[] { 0, 0, 0, 0, -1, 0, 0, 0 });
|
||||
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 });
|
||||
}
|
||||
}
|
|
@ -1,130 +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.stream.io.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.lang.reflect.Field;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class StreamScannerTests {
|
||||
|
||||
@Test
|
||||
public void validateWithMultiByteCharsNoDelimiter() {
|
||||
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, null, 1000);
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertFalse(scanner.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateWithComplexDelimiter() {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY NEW TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY NEWEST TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertFalse(scanner.hasNext());
|
||||
}
|
||||
|
||||
@Test(expected = IllegalStateException.class)
|
||||
public void validateMaxBufferSize() {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 20);
|
||||
assertTrue(scanner.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyScannerHandlesNegativeOneByteInputs() {
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
|
||||
StreamScanner scanner = new StreamScanner(is, "water".getBytes(StandardCharsets.UTF_8), 20, 1024);
|
||||
assertTrue(scanner.hasNext());
|
||||
Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyScannerHandlesNegativeOneByteDelimiter() {
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(new byte[]{0, 0, 0, 0, -1, 0, 0, 0});
|
||||
StreamScanner scanner = new StreamScanner(is, new byte[] { -1 }, 20, 1024);
|
||||
assertTrue(scanner.hasNext());
|
||||
Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0, 0});
|
||||
assertTrue(scanner.hasNext());
|
||||
Assert.assertArrayEquals(scanner.next(), new byte[]{0, 0, 0});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateHasNextIdempotencyWithDelimiter() {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, "<MY DEIMITER>".getBytes(StandardCharsets.UTF_8), 1000);
|
||||
for (int i = 0; i < 5; i++) { // we only have 3 segments so unless idempotent hasNext would return false after 3 tries
|
||||
assertTrue(scanner.hasNext());
|
||||
}
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY NEW TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals("THIS IS MY NEWEST TEXT", new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertFalse(scanner.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateHasNextIdempotencyWithoutDelimiter() {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, null, 1000);
|
||||
for (int i = 0; i < 5; i++) { // we only have 3 segments so unless idempotent hasNext would return false after 3 tries
|
||||
assertTrue(scanner.hasNext());
|
||||
}
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertFalse(scanner.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateInternalBufferCanExpend() throws Exception {
|
||||
String data = "THIS IS MY TEXT<MY DEIMITER>THIS IS MY NEW TEXT<MY DEIMITER>THIS IS MY NEWEST TEXT";
|
||||
ByteArrayInputStream is = new ByteArrayInputStream(data.getBytes());
|
||||
StreamScanner scanner = new StreamScanner(is, null, 1000, 2);
|
||||
Field bufferField = StreamScanner.class.getDeclaredField("buffer");
|
||||
bufferField.setAccessible(true);
|
||||
ByteBuffer buffer = (ByteBuffer) bufferField.get(scanner);
|
||||
assertEquals(2, buffer.capacity());
|
||||
|
||||
assertTrue(scanner.hasNext());
|
||||
assertEquals(data, new String(scanner.next(), StandardCharsets.UTF_8));
|
||||
assertFalse(scanner.hasNext());
|
||||
|
||||
buffer = (ByteBuffer) bufferField.get(scanner);
|
||||
assertEquals(128, buffer.capacity());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,145 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.kafka;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
|
||||
/**
|
||||
* Base class for {@link Processor}s to publish and consume messages from Kafka
|
||||
*
|
||||
* @see PutKafka
|
||||
*/
|
||||
abstract class AbstractKafkaProcessor<T extends Closeable> extends AbstractSessionFactoryProcessor {
|
||||
|
||||
|
||||
private volatile boolean acceptTask = true;
|
||||
|
||||
private final AtomicInteger taskCounter = new AtomicInteger();
|
||||
|
||||
|
||||
/**
|
||||
* @see KafkaPublisher
|
||||
*/
|
||||
volatile T kafkaResource;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@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 'kafkaResource' 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 'kafkaResource' can become null right after its null
|
||||
* check passed causing subsequent NPE.
|
||||
*/
|
||||
synchronized (this) {
|
||||
if (this.kafkaResource == null) {
|
||||
this.kafkaResource = this.buildKafkaResource(context, session);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* 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
|
||||
* - PublishKafka - some messages were sent to Kafka based on existence of the input FlowFile
|
||||
*/
|
||||
boolean processed = this.rendezvousWithKafka(context, session);
|
||||
session.commit();
|
||||
if (processed) {
|
||||
this.postCommit(context);
|
||||
} else {
|
||||
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 {
|
||||
context.yield();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
@OnStopped
|
||||
public void close() {
|
||||
if (this.taskCounter.get() == 0) {
|
||||
try {
|
||||
if (this.kafkaResource != null) {
|
||||
try {
|
||||
this.kafkaResource.close();
|
||||
} catch (Exception e) {
|
||||
this.getLogger().warn("Failed while closing " + this.kafkaResource, e);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
this.kafkaResource = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This operation will be executed after {@link ProcessSession#commit()} has
|
||||
* been called.
|
||||
*/
|
||||
protected void postCommit(ProcessContext context) {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* This operation is called from
|
||||
* {@link #onTrigger(ProcessContext, ProcessSessionFactory)} method and
|
||||
* contains main processing logic for this Processor.
|
||||
*/
|
||||
protected abstract boolean rendezvousWithKafka(ProcessContext context, ProcessSession session);
|
||||
|
||||
/**
|
||||
* Builds target resource for interacting with Kafka. The target resource
|
||||
* could be one of {@link KafkaPublisher} or {@link KafkaConsumer}
|
||||
*/
|
||||
protected abstract T buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException;
|
||||
}
|
|
@ -16,9 +16,9 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kafka;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -27,47 +27,50 @@ 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.ProducerConfig;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
import org.apache.kafka.common.serialization.ByteArraySerializer;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.stream.io.util.StreamScanner;
|
||||
import org.apache.nifi.stream.io.util.StreamDemarcator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import kafka.producer.KeyedMessage;
|
||||
import kafka.producer.Partitioner;
|
||||
|
||||
/**
|
||||
* Wrapper over {@link KafkaProducer} to assist {@link PutKafka} processor with
|
||||
* sending content of {@link FlowFile}s to Kafka.
|
||||
* Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
|
||||
* with sending contents of the {@link FlowFile}s to Kafka.
|
||||
*/
|
||||
class KafkaPublisher implements AutoCloseable {
|
||||
class KafkaPublisher implements Closeable {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(KafkaPublisher.class);
|
||||
|
||||
private final KafkaProducer<byte[], byte[]> producer;
|
||||
private final Producer<byte[], byte[]> kafkaProducer;
|
||||
|
||||
private final Partitioner partitioner;
|
||||
|
||||
private final long ackWaitTime;
|
||||
private long ackWaitTime = 30000;
|
||||
|
||||
private ProcessorLog processLog;
|
||||
|
||||
private final Partitioner partitioner;
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
|
||||
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
|
||||
this.producer = new KafkaProducer<>(kafkaProperties);
|
||||
this.ackWaitTime = Long.parseLong(kafkaProperties.getProperty(ProducerConfig.TIMEOUT_CONFIG)) * 2;
|
||||
this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
|
||||
try {
|
||||
if (kafkaProperties.containsKey("partitioner.class")){
|
||||
if (kafkaProperties.containsKey("partitioner.class")) {
|
||||
this.partitioner = (Partitioner) Class.forName(kafkaProperties.getProperty("partitioner.class")).newInstance();
|
||||
} else {
|
||||
this.partitioner = null;
|
||||
|
@ -78,152 +81,181 @@ class KafkaPublisher implements AutoCloseable {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
for (; (messageBytes = streamTokenizer.nextToken()) != null; tokenCounter++) {
|
||||
if (prevLastAckedMessageIndex < tokenCounter) {
|
||||
Integer partitionId = publishingContext.getPartitionId();
|
||||
if (partitionId == null && publishingContext.getKeyBytes() != null) {
|
||||
partitionId = this.getPartition(publishingContext.getKeyBytes(), publishingContext.getTopic());
|
||||
}
|
||||
ProducerRecord<byte[], byte[]> message =
|
||||
new ProducerRecord<>(publishingContext.getTopic(), publishingContext.getPartitionId(), publishingContext.getKeyBytes(), messageBytes);
|
||||
resultFutures.add(this.kafkaProducer.send(message));
|
||||
}
|
||||
}
|
||||
|
||||
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
|
||||
return new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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}
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
this.kafkaProducer.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Will set {@link ProcessorLog} as an additional logger to forward log
|
||||
* messages to NiFi bulletin
|
||||
*/
|
||||
void setProcessLog(ProcessorLog processLog) {
|
||||
this.processLog = processLog;
|
||||
}
|
||||
|
||||
/**
|
||||
* Publishes messages to Kafka topic. It supports three publishing
|
||||
* mechanisms.
|
||||
* <ul>
|
||||
* <li>Sending the entire content stream as a single Kafka message.</li>
|
||||
* <li>Splitting the incoming content stream into chunks and sending
|
||||
* individual chunks as separate Kafka messages.</li>
|
||||
* <li>Splitting the incoming content stream into chunks and sending only
|
||||
* the chunks that have failed previously @see
|
||||
* {@link SplittableMessageContext#getFailedSegments()}.</li>
|
||||
* </ul>
|
||||
* 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 failed segment list will keep the index of
|
||||
* of each content stream segment that had failed to be sent to Kafka, so
|
||||
* upon retry only the failed segments are sent.
|
||||
*
|
||||
* @param messageContext
|
||||
* instance of {@link SplittableMessageContext} which hold
|
||||
* context information about the message to be sent
|
||||
* @param contentStream
|
||||
* instance of open {@link InputStream} carrying the content of
|
||||
* the message(s) to be send to Kafka
|
||||
* @param partitionKey
|
||||
* the value of the partition key. Only relevant is user wishes
|
||||
* to provide a custom partition key instead of relying on
|
||||
* variety of provided {@link Partitioner}(s)
|
||||
* @param maxBufferSize maximum message size
|
||||
* @return The set containing the failed segment indexes for messages that
|
||||
* failed to be sent to Kafka.
|
||||
*/
|
||||
BitSet publish(SplittableMessageContext messageContext, InputStream contentStream, Integer partitionKey,
|
||||
int maxBufferSize) {
|
||||
List<Future<RecordMetadata>> sendFutures = this.split(messageContext, contentStream, partitionKey, maxBufferSize);
|
||||
return this.publish(sendFutures);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method splits (if required) the incoming content stream into
|
||||
* messages to publish to Kafka topic. See publish method for more
|
||||
* details
|
||||
*
|
||||
* @param messageContext
|
||||
* instance of {@link SplittableMessageContext} which hold
|
||||
* context information about the message to be sent
|
||||
* @param contentStream
|
||||
* instance of open {@link InputStream} carrying the content of
|
||||
* the message(s) to be send to Kafka
|
||||
* @param partitionKey
|
||||
* the value of the partition key. Only relevant is user wishes
|
||||
* to provide a custom partition key instead of relying on
|
||||
* variety of provided {@link Partitioner}(s)
|
||||
* @param maxBufferSize maximum message size
|
||||
* @return The list of messages to publish
|
||||
*/
|
||||
List<Future<RecordMetadata>> split(SplittableMessageContext messageContext, InputStream contentStream, Integer partitionKey,
|
||||
int maxBufferSize) {
|
||||
List<Future<RecordMetadata>> sendFutures = new ArrayList<>();
|
||||
BitSet prevFailedSegmentIndexes = messageContext.getFailedSegments();
|
||||
int segmentCounter = 0;
|
||||
StreamScanner scanner = new StreamScanner(contentStream, messageContext.getDelimiterBytes(), maxBufferSize);
|
||||
|
||||
while (scanner.hasNext()) {
|
||||
byte[] content = scanner.next();
|
||||
if (content.length > 0){
|
||||
byte[] key = messageContext.getKeyBytes();
|
||||
String topicName = messageContext.getTopicName();
|
||||
if (partitionKey == null && key != null) {
|
||||
partitionKey = this.getPartition(key, topicName);
|
||||
}
|
||||
if (prevFailedSegmentIndexes == null || prevFailedSegmentIndexes.get(segmentCounter)) {
|
||||
ProducerRecord<byte[], byte[]> message = new ProducerRecord<>(topicName, partitionKey, key, content);
|
||||
sendFutures.add(this.toKafka(message));
|
||||
}
|
||||
segmentCounter++;
|
||||
private void warnOrError(String message, Exception e) {
|
||||
if (e == null) {
|
||||
logger.warn(message);
|
||||
if (this.processLog != null) {
|
||||
this.processLog.warn(message);
|
||||
}
|
||||
} else {
|
||||
logger.error(message, e);
|
||||
if (this.processLog != null) {
|
||||
this.processLog.error(message, e);
|
||||
}
|
||||
}
|
||||
return sendFutures;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
BitSet publish(List<Future<RecordMetadata>> sendFutures) {
|
||||
int segmentCounter = 0;
|
||||
BitSet failedSegments = new BitSet();
|
||||
for (Future<RecordMetadata> future : sendFutures) {
|
||||
try {
|
||||
future.get(this.ackWaitTime, TimeUnit.MILLISECONDS);
|
||||
} catch (InterruptedException e) {
|
||||
failedSegments.set(segmentCounter);
|
||||
Thread.currentThread().interrupt();
|
||||
logger.warn("Interrupted while waiting for acks from Kafka");
|
||||
if (this.processLog != null) {
|
||||
this.processLog.warn("Interrupted while waiting for acks from Kafka");
|
||||
}
|
||||
} catch (ExecutionException e) {
|
||||
failedSegments.set(segmentCounter);
|
||||
logger.error("Failed while waiting for acks from Kafka", e);
|
||||
if (this.processLog != null) {
|
||||
this.processLog.error("Failed while waiting for acks from Kafka", e);
|
||||
}
|
||||
} catch (TimeoutException e) {
|
||||
failedSegments.set(segmentCounter);
|
||||
logger.warn("Timed out while waiting for acks from Kafka");
|
||||
if (this.processLog != null) {
|
||||
this.processLog.warn("Timed out while waiting for acks from Kafka");
|
||||
}
|
||||
}
|
||||
segmentCounter++;
|
||||
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.messagesSent - 1 == this.lastMessageAcked;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
|
||||
}
|
||||
return failedSegments;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private int getPartition(Object key, String topicName) {
|
||||
int partSize = this.producer.partitionsFor(topicName).size();
|
||||
return this.partitioner.partition(key, partSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes {@link KafkaProducer}
|
||||
*/
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
this.producer.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the provided {@link KeyedMessage} to Kafka async returning
|
||||
* {@link Future}
|
||||
*/
|
||||
private Future<RecordMetadata> toKafka(ProducerRecord<byte[], byte[]> message) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Publishing message to '" + message.topic() + "' topic.");
|
||||
if (this.partitioner != null) {
|
||||
int partSize = this.kafkaProducer.partitionsFor(topicName).size();
|
||||
return this.partitioner.partition(key, partSize);
|
||||
}
|
||||
return this.producer.send(message);
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,12 +40,11 @@ final public class Partitioners {
|
|||
return partitionIndex;
|
||||
}
|
||||
|
||||
private int next(int numberOfPartitions) {
|
||||
if (index == numberOfPartitions) {
|
||||
index = 0;
|
||||
private synchronized int next(int numberOfPartitions) {
|
||||
if (this.index >= numberOfPartitions) {
|
||||
this.index = 0;
|
||||
}
|
||||
int indexToReturn = index++;
|
||||
return indexToReturn;
|
||||
return index++;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.kafka;
|
||||
|
||||
import 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 volatile Integer partitionId;
|
||||
|
||||
/*
|
||||
* We're using the default value from Kafka. We are using it to control the
|
||||
* message size before it goes to to Kafka thus limiting possibility of a
|
||||
* late failures in Kafka client.
|
||||
*/
|
||||
private volatile int maxRequestSize = 1048576; // kafka default
|
||||
|
||||
private volatile boolean maxRequestSizeSet;
|
||||
|
||||
private volatile byte[] keyBytes;
|
||||
|
||||
private volatile byte[] delimiterBytes;
|
||||
|
||||
|
||||
|
||||
PublishingContext(InputStream contentStream, String topic) {
|
||||
this(contentStream, topic, -1);
|
||||
}
|
||||
|
||||
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
|
||||
this.validateInput(contentStream, topic, lastAckedMessageIndex);
|
||||
this.contentStream = contentStream;
|
||||
this.topic = topic;
|
||||
this.lastAckedMessageIndex = lastAckedMessageIndex;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
Integer getPartitionId() {
|
||||
return partitionId;
|
||||
}
|
||||
|
||||
public void setPartitionId(Integer partitionId) {
|
||||
this.partitionId = partitionId;
|
||||
}
|
||||
|
||||
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");
|
||||
}
|
||||
}
|
||||
|
||||
void setMaxRequestSize(int maxRequestSize) {
|
||||
if (!this.maxRequestSizeSet) {
|
||||
if (maxRequestSize > 0) {
|
||||
this.maxRequestSize = maxRequestSize;
|
||||
this.maxRequestSizeSet = true;
|
||||
} else {
|
||||
throw new IllegalArgumentException("'maxRequestSize' must be > 0");
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("'maxRequestSize' 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");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -30,25 +29,20 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
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.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
|
@ -56,7 +50,7 @@ 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.StandardValidators;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
import org.apache.nifi.processors.kafka.KafkaPublisher.KafkaPublisherResult;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" })
|
||||
|
@ -67,7 +61,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
+ " In the event a dynamic property represents a property that was already set as part of the static properties, its value wil be"
|
||||
+ " overriden with warning message describing the override."
|
||||
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.")
|
||||
public class PutKafka extends AbstractProcessor {
|
||||
public class PutKafka extends AbstractKafkaProcessor<KafkaPublisher> {
|
||||
|
||||
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
|
||||
|
||||
|
@ -162,9 +156,9 @@ public class PutKafka extends AbstractProcessor {
|
|||
+ "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. Note that if messages are delimited and some messages for a given FlowFile are transferred "
|
||||
+ "successfully while others are not, the FlowFile will be transferred to the 'failure' relationship. In "
|
||||
+ "case the FlowFile is sent back to this processor, only the messages not previously transferred "
|
||||
+ "successfully will be handled by the processor to be retransferred to Kafka.")
|
||||
+ "successfully while others are not, the messages will be split into individual FlowFiles, such that those "
|
||||
+ "messages that were successfully sent are routed to the 'success' relationship while other messages are "
|
||||
+ "sent to the 'failure' relationship.")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
|
@ -199,13 +193,14 @@ public class PutKafka extends AbstractProcessor {
|
|||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
public static final PropertyDescriptor BATCH_NUM_MESSAGES = new PropertyDescriptor.Builder()
|
||||
.name("Async Batch Size").displayName("Batch Size")
|
||||
.description("The number of messages to send in one batch. The producer will wait until either this number of messages are ready "
|
||||
+ "to send or \"Queue Buffering Max Time\" is reached. NOTE: This property will be ignored unless the 'Message Delimiter' "
|
||||
+ "property is specified.")
|
||||
.name("Async Batch Size")
|
||||
.displayName("Batch Size")
|
||||
.description("This configuration controls the default batch size in bytes.The producer will attempt to batch records together into "
|
||||
+ "fewer requests whenever multiple records are being sent to the same partition. This helps performance on both the client "
|
||||
+ "and the server.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("200")
|
||||
.defaultValue("16384") // Kafka default
|
||||
.build();
|
||||
public static final PropertyDescriptor QUEUE_BUFFERING_MAX = new PropertyDescriptor.Builder()
|
||||
.name("Queue Buffering Max Time")
|
||||
|
@ -236,17 +231,15 @@ public class PutKafka extends AbstractProcessor {
|
|||
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
|
||||
protected static final String ATTR_PROC_ID = "PROC_ID";
|
||||
protected static final String FAILED_PROC_ID_ATTR = "failed.proc.id";
|
||||
|
||||
protected static final String ATTR_FAILED_SEGMENTS = "FS";
|
||||
protected static final String FAILED_LAST_ACK_IDX = "failed.last.idx";
|
||||
|
||||
protected static final String ATTR_TOPIC = "TOPIC";
|
||||
protected static final String FAILED_TOPIC_ATTR = "failed.topic";
|
||||
|
||||
protected static final String ATTR_KEY = "KEY";
|
||||
protected static final String FAILED_KEY_ATTR = "failed.key";
|
||||
|
||||
protected static final String ATTR_DELIMITER = "DELIMITER";
|
||||
|
||||
private volatile KafkaPublisher kafkaPublisher;
|
||||
protected static final String FAILED_DELIMITER_ATTR = "failed.delimiter";
|
||||
|
||||
private static final List<PropertyDescriptor> propertyDescriptors;
|
||||
|
||||
|
@ -276,66 +269,117 @@ public class PutKafka extends AbstractProcessor {
|
|||
relationships = Collections.unmodifiableSet(_relationships);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@OnScheduled
|
||||
public void createKafkaPublisher(ProcessContext context) {
|
||||
this.kafkaPublisher = new KafkaPublisher(this.buildKafkaConfigProperties(context));
|
||||
this.kafkaPublisher.setProcessLog(this.getLogger());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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}
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
boolean processed = false;
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile != null) {
|
||||
final SplittableMessageContext messageContext = this.buildMessageContext(flowFile, context, session);
|
||||
final Integer partitionKey = this.determinePartition(messageContext, context, flowFile);
|
||||
final AtomicReference<BitSet> failedSegmentsRef = new AtomicReference<BitSet>();
|
||||
final List<Future<RecordMetadata>> sendFutures = new ArrayList<>();
|
||||
|
||||
StopWatch timer = new StopWatch(true);
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
public void process(InputStream contentStream) throws IOException {
|
||||
int maxRecordSize = context.getProperty(MAX_RECORD_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
sendFutures.addAll(kafkaPublisher.split(messageContext, contentStream, partitionKey, maxRecordSize));
|
||||
failedSegmentsRef.set(kafkaPublisher.publish(sendFutures));
|
||||
}
|
||||
});
|
||||
timer.stop();
|
||||
|
||||
final long duration = timer.getDuration(TimeUnit.MILLISECONDS);
|
||||
final int messagesToSend = sendFutures.size();
|
||||
final int messagesSent = messagesToSend - failedSegmentsRef.get().cardinality();
|
||||
final String details = messagesSent + " message(s) over " + messagesToSend + " sent successfully";
|
||||
if (failedSegmentsRef.get().isEmpty()) {
|
||||
session.getProvenanceReporter().send(flowFile, "kafka://" + context.getProperty(SEED_BROKERS).getValue() + "/" + messageContext.getTopicName(), details, duration);
|
||||
flowFile = this.cleanUpFlowFileIfNecessary(flowFile, session);
|
||||
flowFile = this.doRendezvousWithKafka(flowFile, context, session);
|
||||
if (!this.isFailedFlowFile(flowFile)) {
|
||||
session.getProvenanceReporter().send(flowFile,
|
||||
context.getProperty(SEED_BROKERS).getValue() + "/"
|
||||
+ context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue());
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
} else {
|
||||
if(messagesSent != 0) {
|
||||
session.getProvenanceReporter().send(flowFile, "kafka://" + context.getProperty(SEED_BROKERS).getValue() + "/" + messageContext.getTopicName(), details, duration);
|
||||
}
|
||||
flowFile = session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(failedSegmentsRef.get(), messageContext));
|
||||
session.transfer(session.penalize(flowFile), REL_FAILURE);
|
||||
}
|
||||
|
||||
} else {
|
||||
context.yield();
|
||||
processed = true;
|
||||
}
|
||||
return processed;
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void cleanup() {
|
||||
try {
|
||||
this.kafkaPublisher.close();
|
||||
} catch (Exception e) {
|
||||
getLogger().warn("Failed while closing KafkaPublisher", e);
|
||||
/**
|
||||
* 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<KafkaPublisherResult> publishResultRef = new AtomicReference<>();
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
public void process(InputStream contentStream) throws IOException {
|
||||
PublishingContext publishingContext = PutKafka.this.buildPublishingContext(flowFile, context, contentStream);
|
||||
KafkaPublisherResult result = PutKafka.this.kafkaResource.publish(publishingContext);
|
||||
publishResultRef.set(result);
|
||||
}
|
||||
});
|
||||
|
||||
FlowFile resultFile = publishResultRef.get().isAllAcked()
|
||||
? this.cleanUpFlowFileIfNecessary(flowFile, session)
|
||||
: session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context));
|
||||
|
||||
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 PublishKafka processor it is treated as a fresh FlowFile
|
||||
* regardless if it has #FAILED* attributes set.
|
||||
*/
|
||||
private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context,
|
||||
InputStream contentStream) {
|
||||
String topicName;
|
||||
byte[] keyBytes;
|
||||
byte[] delimiterBytes = null;
|
||||
int lastAckedMessageIndex = -1;
|
||||
if (this.isFailedFlowFile(flowFile)) {
|
||||
lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
|
||||
topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
|
||||
keyBytes = flowFile.getAttribute(FAILED_KEY_ATTR) != null
|
||||
? flowFile.getAttribute(FAILED_KEY_ATTR).getBytes(StandardCharsets.UTF_8) : null;
|
||||
delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
|
||||
? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
|
||||
|
||||
} else {
|
||||
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
||||
String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||
keyBytes = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8);
|
||||
delimiterBytes = context.getProperty(MESSAGE_DELIMITER).isSet() ? context.getProperty(MESSAGE_DELIMITER)
|
||||
.evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
|
||||
}
|
||||
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
|
||||
publishingContext.setKeyBytes(keyBytes);
|
||||
publishingContext.setDelimiterBytes(delimiterBytes);
|
||||
publishingContext.setPartitionId(this.determinePartition(context, flowFile));
|
||||
return publishingContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -343,6 +387,14 @@ public class PutKafka extends AbstractProcessor {
|
|||
return relationships;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
|
||||
throws ProcessException {
|
||||
KafkaPublisher kafkaPublisher = new KafkaPublisher(this.buildKafkaConfigProperties(context));
|
||||
kafkaPublisher.setProcessLog(this.getLogger());
|
||||
return kafkaPublisher;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
|
@ -374,12 +426,14 @@ public class PutKafka extends AbstractProcessor {
|
|||
*
|
||||
*/
|
||||
private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) {
|
||||
if (flowFile.getAttribute(ATTR_FAILED_SEGMENTS) != null) {
|
||||
flowFile = session.removeAttribute(flowFile, ATTR_FAILED_SEGMENTS);
|
||||
flowFile = session.removeAttribute(flowFile, ATTR_KEY);
|
||||
flowFile = session.removeAttribute(flowFile, ATTR_TOPIC);
|
||||
flowFile = session.removeAttribute(flowFile, ATTR_DELIMITER);
|
||||
flowFile = session.removeAttribute(flowFile, ATTR_PROC_ID);
|
||||
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;
|
||||
}
|
||||
|
@ -387,7 +441,7 @@ public class PutKafka extends AbstractProcessor {
|
|||
/**
|
||||
*
|
||||
*/
|
||||
private Integer determinePartition(SplittableMessageContext messageContext, ProcessContext context, FlowFile flowFile) {
|
||||
private Integer determinePartition(ProcessContext context, FlowFile flowFile) {
|
||||
String partitionStrategy = context.getProperty(PARTITION_STRATEGY).getValue();
|
||||
Integer partitionValue = null;
|
||||
if (partitionStrategy.equalsIgnoreCase(USER_DEFINED_PARTITIONING.getValue())) {
|
||||
|
@ -400,46 +454,27 @@ public class PutKafka extends AbstractProcessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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(BitSet failedSegments, SplittableMessageContext messageContext) {
|
||||
private Map<String, String> buildFailedFlowFileAttributes(int lastAckedMessageIndex, FlowFile sourceFlowFile,
|
||||
ProcessContext context) {
|
||||
Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put(ATTR_PROC_ID, this.getIdentifier());
|
||||
attributes.put(ATTR_FAILED_SEGMENTS, new String(failedSegments.toByteArray(), StandardCharsets.UTF_8));
|
||||
attributes.put(ATTR_TOPIC, messageContext.getTopicName());
|
||||
attributes.put(ATTR_KEY, messageContext.getKeyBytesAsString());
|
||||
attributes.put(ATTR_DELIMITER, new String(messageContext.getDelimiterBytes(), StandardCharsets.UTF_8));
|
||||
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_DELIMITER).isSet()
|
||||
? context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(sourceFlowFile).getValue()
|
||||
: null);
|
||||
return attributes;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private SplittableMessageContext buildMessageContext(FlowFile flowFile, ProcessContext context, ProcessSession session) {
|
||||
String topicName;
|
||||
byte[] key;
|
||||
byte[] delimiterBytes;
|
||||
|
||||
String failedSegmentsString = flowFile.getAttribute(ATTR_FAILED_SEGMENTS);
|
||||
if (flowFile.getAttribute(ATTR_PROC_ID) != null && flowFile.getAttribute(ATTR_PROC_ID).equals(this.getIdentifier()) && failedSegmentsString != null) {
|
||||
topicName = flowFile.getAttribute(ATTR_TOPIC);
|
||||
key = flowFile.getAttribute(ATTR_KEY) == null ? null : flowFile.getAttribute(ATTR_KEY).getBytes();
|
||||
delimiterBytes = flowFile.getAttribute(ATTR_DELIMITER) != null ? flowFile.getAttribute(ATTR_DELIMITER).getBytes(StandardCharsets.UTF_8) : null;
|
||||
} else {
|
||||
failedSegmentsString = null;
|
||||
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
||||
String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||
key = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8);
|
||||
delimiterBytes = context.getProperty(MESSAGE_DELIMITER).isSet()
|
||||
? context.getProperty(MESSAGE_DELIMITER).evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
|
||||
}
|
||||
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, key, delimiterBytes);
|
||||
if (failedSegmentsString != null) {
|
||||
messageContext.setFailedSegmentsAsByteArray(failedSegmentsString.getBytes());
|
||||
}
|
||||
return messageContext;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -450,11 +485,7 @@ public class PutKafka extends AbstractProcessor {
|
|||
properties.setProperty("acks", context.getProperty(DELIVERY_GUARANTEE).getValue());
|
||||
properties.setProperty("buffer.memory", String.valueOf(context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue()));
|
||||
properties.setProperty("compression.type", context.getProperty(COMPRESSION_CODEC).getValue());
|
||||
if (context.getProperty(MESSAGE_DELIMITER).isSet()) {
|
||||
properties.setProperty("batch.size", context.getProperty(BATCH_NUM_MESSAGES).getValue());
|
||||
} else {
|
||||
properties.setProperty("batch.size", "1");
|
||||
}
|
||||
properties.setProperty("batch.size", context.getProperty(BATCH_NUM_MESSAGES).getValue());
|
||||
|
||||
properties.setProperty("client.id", context.getProperty(CLIENT_NAME).getValue());
|
||||
Long queueBufferingMillis = context.getProperty(QUEUE_BUFFERING_MAX).asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
|
|
|
@ -1,123 +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;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.BitSet;
|
||||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
|
||||
/**
|
||||
* Context object that serves as a bridge between the content of a FlowFile and
|
||||
* Kafka message(s). It contains all necessary information to allow
|
||||
* {@link KafkaPublisher} to determine how a each content of the
|
||||
* {@link FlowFile} must be sent to Kafka.
|
||||
*/
|
||||
final class SplittableMessageContext {
|
||||
private final String topicName;
|
||||
|
||||
private final byte[] delimiterBytes;
|
||||
|
||||
private final byte[] keyBytes;
|
||||
|
||||
private volatile BitSet failedSegments;
|
||||
|
||||
/**
|
||||
* @param topicName
|
||||
* the name of the Kafka topic
|
||||
* @param keyBytes
|
||||
* the instance of byte[] representing the key. Can be null.
|
||||
* @param delimiterBytes
|
||||
* byte array representing bytes by which the data will be
|
||||
* delimited. Can be null.
|
||||
*/
|
||||
SplittableMessageContext(String topicName, byte[] keyBytes, byte[] delimiterBytes) {
|
||||
if (topicName == null || topicName.trim().length() == 0){
|
||||
throw new IllegalArgumentException("'topicName' must not be null or empty");
|
||||
}
|
||||
this.topicName = topicName;
|
||||
this.keyBytes = keyBytes;
|
||||
this.delimiterBytes = delimiterBytes != null ? delimiterBytes : null;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
String delVal = this.delimiterBytes != null ? " delimiter: '" + new String(this.delimiterBytes, StandardCharsets.UTF_8) + "'" : "";
|
||||
return "topic: '" + topicName + "';" + delVal;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will set failed segments from an array of integers
|
||||
*/
|
||||
void setFailedSegments(int... failedSegments) {
|
||||
if (failedSegments != null) {
|
||||
this.failedSegments = new BitSet();
|
||||
for (int failedSegment : failedSegments) {
|
||||
this.failedSegments.set(failedSegment);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will set failed segments from an array of bytes that will be used to
|
||||
* construct the final {@link BitSet} representing failed segments
|
||||
*/
|
||||
void setFailedSegmentsAsByteArray(byte[] failedSegments) {
|
||||
if (failedSegments != null) {
|
||||
this.failedSegments = BitSet.valueOf(failedSegments);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the list of integers representing the segments (chunks) of the
|
||||
* delimited content stream that had failed to be sent to Kafka topic.
|
||||
*/
|
||||
BitSet getFailedSegments() {
|
||||
return this.failedSegments;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the name of the Kafka topic
|
||||
*/
|
||||
String getTopicName() {
|
||||
return this.topicName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the delimiter bytes
|
||||
*/
|
||||
byte[] getDelimiterBytes() {
|
||||
return this.delimiterBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the key bytes as String
|
||||
*/
|
||||
String getKeyBytesAsString() {
|
||||
return this.keyBytes != null ? new String(this.keyBytes) : null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the key bytes
|
||||
*/
|
||||
byte[] getKeyBytes() {
|
||||
return this.keyBytes;
|
||||
}
|
||||
}
|
|
@ -28,8 +28,12 @@ import org.apache.nifi.util.TestRunner;
|
|||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
@Ignore
|
||||
// The test is valid and should be ran when working on this module. @Ignore is
|
||||
// to speed up the overall build
|
||||
public class GetKafkaIntegrationTests {
|
||||
|
||||
private static EmbeddedKafka kafkaLocal;
|
||||
|
|
|
@ -28,6 +28,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.kafka.common.serialization.ByteArraySerializer;
|
||||
import org.apache.nifi.processors.kafka.KafkaPublisher.KafkaPublisherResult;
|
||||
import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
|
||||
import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -41,6 +43,8 @@ import kafka.consumer.ConsumerTimeoutException;
|
|||
import kafka.consumer.KafkaStream;
|
||||
import kafka.javaapi.consumer.ConsumerConnector;
|
||||
|
||||
// The test is valid and should be ran when working on this module. @Ignore is
|
||||
// to speed up the overall build
|
||||
public class KafkaPublisherTest {
|
||||
|
||||
private static EmbeddedKafka kafkaLocal;
|
||||
|
@ -62,17 +66,18 @@ public class KafkaPublisherTest {
|
|||
|
||||
@Test
|
||||
public void validateSuccessfulSendAsWhole() throws Exception {
|
||||
InputStream fis = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
|
||||
InputStream contentStream = new ByteArrayInputStream("Hello Kafka".getBytes(StandardCharsets.UTF_8));
|
||||
String topicName = "validateSuccessfulSendAsWhole";
|
||||
|
||||
Properties kafkaProperties = this.buildProducerProperties();
|
||||
KafkaPublisher publisher = new KafkaPublisher(kafkaProperties);
|
||||
|
||||
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null);
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
|
||||
KafkaPublisherResult result = publisher.publish(publishingContext);
|
||||
|
||||
publisher.publish(messageContext, fis, null, 2000);
|
||||
|
||||
fis.close();
|
||||
assertEquals(0, result.getLastMessageAcked());
|
||||
assertEquals(1, result.getMessagesSent());
|
||||
contentStream.close();
|
||||
publisher.close();
|
||||
|
||||
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
|
||||
|
@ -86,16 +91,20 @@ public class KafkaPublisherTest {
|
|||
|
||||
@Test
|
||||
public void validateSuccessfulSendAsDelimited() throws Exception {
|
||||
InputStream fis = new ByteArrayInputStream(
|
||||
"Hello Kafka 1\nHello Kafka 2\nHello Kafka 3\nHello Kafka 4\n".getBytes(StandardCharsets.UTF_8));
|
||||
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);
|
||||
|
||||
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n".getBytes(StandardCharsets.UTF_8));
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
|
||||
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
|
||||
KafkaPublisherResult result = publisher.publish(publishingContext);
|
||||
|
||||
publisher.publish(messageContext, fis, null, 2000);
|
||||
assertEquals(3, result.getLastMessageAcked());
|
||||
assertEquals(4, result.getMessagesSent());
|
||||
contentStream.close();
|
||||
publisher.close();
|
||||
|
||||
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
|
||||
|
@ -111,48 +120,111 @@ public class KafkaPublisherTest {
|
|||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* This test simulates the condition where not all messages were ACKed by
|
||||
* Kafka
|
||||
*/
|
||||
@Test
|
||||
public void validateSuccessfulReSendOfFailedSegments() throws Exception {
|
||||
InputStream fis = new ByteArrayInputStream(
|
||||
"Hello Kafka 1\nHello Kafka 2\nHello Kafka 3\nHello Kafka 4\n".getBytes(StandardCharsets.UTF_8));
|
||||
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);
|
||||
|
||||
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, "\n".getBytes(StandardCharsets.UTF_8));
|
||||
messageContext.setFailedSegments(1, 3);
|
||||
// simulates the first re-try
|
||||
int lastAckedMessageIndex = 1;
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
|
||||
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
|
||||
|
||||
publisher.publish(messageContext, fis, null, 2000);
|
||||
publisher.close();
|
||||
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 Kafka 2", m1);
|
||||
assertEquals("Hello Kafka 4", m2);
|
||||
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 validateWithMultiByteCharacters() throws Exception {
|
||||
String data = "僠THIS IS MY NEW TEXT.僠IT HAS A NEWLINE.";
|
||||
InputStream fis = new ByteArrayInputStream(data.getBytes(StandardCharsets.UTF_8));
|
||||
String topicName = "validateWithMultiByteCharacters";
|
||||
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);
|
||||
|
||||
SplittableMessageContext messageContext = new SplittableMessageContext(topicName, null, null);
|
||||
// simulates the first re-try
|
||||
int lastAckedMessageIndex = 3;
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
|
||||
publishingContext.setDelimiterBytes("\n".getBytes(StandardCharsets.UTF_8));
|
||||
|
||||
publisher.publish(messageContext, fis, null, 2000);
|
||||
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);
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName);
|
||||
|
||||
publisher.publish(publishingContext);
|
||||
publisher.close();
|
||||
|
||||
ConsumerIterator<byte[], byte[]> iter = this.buildConsumer(topicName);
|
||||
|
@ -162,12 +234,10 @@ public class KafkaPublisherTest {
|
|||
|
||||
private Properties buildProducerProperties() {
|
||||
Properties kafkaProperties = new Properties();
|
||||
kafkaProperties.setProperty("bootstrap.servers", "0.0.0.0:" + kafkaLocal.getKafkaPort());
|
||||
kafkaProperties.setProperty("serializer.class", "kafka.serializer.DefaultEncoder");
|
||||
kafkaProperties.setProperty("acks", "1");
|
||||
kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName());
|
||||
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
|
||||
kafkaProperties.setProperty("bootstrap.servers", "localhost:" + kafkaLocal.getKafkaPort());
|
||||
kafkaProperties.put("auto.create.topics.enable", "true");
|
||||
kafkaProperties.setProperty("partitioner.class", "org.apache.nifi.processors.kafka.Partitioners$RoundRobinPartitioner");
|
||||
kafkaProperties.setProperty("timeout.ms", "5000");
|
||||
return kafkaProperties;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,12 +18,9 @@ package org.apache.nifi.processors.kafka;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.BitSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -31,12 +28,10 @@ import java.util.Properties;
|
|||
|
||||
import org.apache.nifi.processors.kafka.test.EmbeddedKafka;
|
||||
import org.apache.nifi.processors.kafka.test.EmbeddedKafkaProducerHelper;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import kafka.consumer.Consumer;
|
||||
|
@ -45,15 +40,16 @@ import kafka.consumer.ConsumerIterator;
|
|||
import kafka.consumer.KafkaStream;
|
||||
import kafka.javaapi.consumer.ConsumerConnector;
|
||||
|
||||
|
||||
public class TestPutKafka {
|
||||
// The test is valid and should be ran when working on this module. @Ignore is
|
||||
// to speed up the overall build
|
||||
public class PutKafkaTest {
|
||||
|
||||
private static EmbeddedKafka kafkaLocal;
|
||||
|
||||
private static EmbeddedKafkaProducerHelper producerHelper;
|
||||
|
||||
@BeforeClass
|
||||
public static void bforeClass() {
|
||||
public static void beforeClass() {
|
||||
kafkaLocal = new EmbeddedKafka();
|
||||
kafkaLocal.start();
|
||||
producerHelper = new EmbeddedKafkaProducerHelper(kafkaLocal);
|
||||
|
@ -66,9 +62,8 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void testDelimitedMessagesWithKey() {
|
||||
String topicName = "testDelimitedMessagesWithKey";
|
||||
public void validateSingleCharacterDemarcatedMessages() {
|
||||
String topicName = "validateSingleCharacterDemarcatedMessages";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
||||
|
@ -94,73 +89,35 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void testWithFailureAndPartialResend() throws Exception {
|
||||
String topicName = "testWithFailureAndPartialResend";
|
||||
public void validateMultiCharacterDelimiyedMessages() {
|
||||
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
final TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
||||
runner.setProperty(PutKafka.CLIENT_NAME, "foo");
|
||||
runner.setProperty(PutKafka.KEY, "key1");
|
||||
runner.setProperty(PutKafka.SEED_BROKERS, "0.0.0.0:" + kafkaLocal.getKafkaPort());
|
||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\n");
|
||||
|
||||
final String text = "Hello World\nGoodbye\n1\n2";
|
||||
runner.enqueue(text.getBytes(StandardCharsets.UTF_8));
|
||||
afterClass(); // kill Kafka right before send to ensure producer fails
|
||||
runner.run(1, false);
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_FAILURE, 1);
|
||||
MockFlowFile ff = runner.getFlowFilesForRelationship(PutKafka.REL_FAILURE).get(0);
|
||||
String failedSegmentsStr = ff.getAttribute(PutKafka.ATTR_FAILED_SEGMENTS);
|
||||
BitSet fs = BitSet.valueOf(failedSegmentsStr.getBytes(StandardCharsets.UTF_8));
|
||||
assertTrue(fs.get(0));
|
||||
assertTrue(fs.get(1));
|
||||
assertTrue(fs.get(2));
|
||||
assertTrue(fs.get(3));
|
||||
String delimiter = ff.getAttribute(PutKafka.ATTR_DELIMITER);
|
||||
assertEquals("\n", delimiter);
|
||||
String key = ff.getAttribute(PutKafka.ATTR_KEY);
|
||||
assertEquals("key1", key);
|
||||
String topic = ff.getAttribute(PutKafka.ATTR_TOPIC);
|
||||
assertEquals(topicName, topic);
|
||||
|
||||
bforeClass();
|
||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:" + kafkaLocal.getKafkaPort());
|
||||
Map<String, String> attr = new HashMap<>(ff.getAttributes());
|
||||
/*
|
||||
* So here we are emulating partial success. Basically even though all 4
|
||||
* messages failed to be sent by changing the ATTR_FAILED_SEGMENTS value
|
||||
* we essentially saying that only two failed and need to be resent.
|
||||
*/
|
||||
BitSet _fs = new BitSet();
|
||||
_fs.set(1);
|
||||
_fs.set(3);
|
||||
attr.put(PutKafka.ATTR_FAILED_SEGMENTS, new String(_fs.toByteArray(), StandardCharsets.UTF_8));
|
||||
ff.putAttributes(attr);
|
||||
runner.enqueue(ff);
|
||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "foo");
|
||||
|
||||
runner.enqueue("Hello WorldfooGoodbyefoo1foo2foo3foo4foo5".getBytes(StandardCharsets.UTF_8));
|
||||
runner.run(1, false);
|
||||
MockFlowFile sff = runner.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0);
|
||||
assertNull(sff.getAttribute(PutKafka.ATTR_FAILED_SEGMENTS));
|
||||
assertNull(sff.getAttribute(PutKafka.ATTR_TOPIC));
|
||||
assertNull(sff.getAttribute(PutKafka.ATTR_KEY));
|
||||
assertNull(sff.getAttribute(PutKafka.ATTR_DELIMITER));
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
||||
ConsumerIterator<byte[], byte[]> consumer = this.buildConsumer(topicName);
|
||||
|
||||
assertEquals("Hello World", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
assertEquals("Goodbye", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
assertEquals("1", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
assertEquals("2", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
try {
|
||||
consumer.next();
|
||||
fail();
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
assertEquals("3", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
assertEquals("4", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
assertEquals("5", new String(consumer.next().message(), StandardCharsets.UTF_8));
|
||||
|
||||
runner.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithEmptyMessages() {
|
||||
String topicName = "testWithEmptyMessages";
|
||||
public void validateDemarcationIntoEmptyMessages() {
|
||||
String topicName = "validateDemarcationIntoEmptyMessages";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
final TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
||||
|
@ -169,13 +126,14 @@ public class TestPutKafka {
|
|||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:" + kafkaLocal.getKafkaPort());
|
||||
runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\n");
|
||||
|
||||
final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
|
||||
final byte[] bytes = "\n\n\n1\n2\n\n\n3\n4\n\n\n".getBytes(StandardCharsets.UTF_8);
|
||||
runner.enqueue(bytes);
|
||||
runner.run(1);
|
||||
|
||||
runner.assertAllFlowFilesTransferred(PutKafka.REL_SUCCESS, 1);
|
||||
|
||||
ConsumerIterator<byte[], byte[]> consumer = this.buildConsumer(topicName);
|
||||
|
||||
assertNotNull(consumer.next());
|
||||
assertNotNull(consumer.next());
|
||||
assertNotNull(consumer.next());
|
||||
|
@ -189,8 +147,8 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testComplexRightPartialDelimitedMessages() {
|
||||
String topicName = "testComplexRightPartialDelimitedMessages";
|
||||
public void validateComplexRightPartialDemarcatedMessages() {
|
||||
String topicName = "validateComplexRightPartialDemarcatedMessages";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
||||
|
@ -210,8 +168,8 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testComplexLeftPartialDelimitedMessages() {
|
||||
String topicName = "testComplexLeftPartialDelimitedMessages";
|
||||
public void validateComplexLeftPartialDemarcatedMessages() {
|
||||
String topicName = "validateComplexLeftPartialDemarcatedMessages";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
||||
|
@ -233,8 +191,8 @@ public class TestPutKafka {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testComplexPartialMatchDelimitedMessages() {
|
||||
String topicName = "testComplexPartialMatchDelimitedMessages";
|
||||
public void validateComplexPartialMatchDemarcatedMessages() {
|
||||
String topicName = "validateComplexPartialMatchDemarcatedMessages";
|
||||
PutKafka putKafka = new PutKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PutKafka.TOPIC, topicName);
|
|
@ -1,66 +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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class SplittableMessageContextTest {
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void failNullEmptyTopic() {
|
||||
new SplittableMessageContext(null, null, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateFullSetting() {
|
||||
SplittableMessageContext ctx = new SplittableMessageContext("foo", "hello".getBytes(), "\n".getBytes(StandardCharsets.UTF_8));
|
||||
ctx.setFailedSegments(1, 3, 6);
|
||||
assertEquals("\n", new String(ctx.getDelimiterBytes(), StandardCharsets.UTF_8));
|
||||
assertEquals("hello", new String(ctx.getKeyBytes(), StandardCharsets.UTF_8));
|
||||
assertEquals("foo", ctx.getTopicName());
|
||||
assertEquals("topic: 'foo'; delimiter: '\n'", ctx.toString());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void validateToString() {
|
||||
SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null);
|
||||
assertEquals("topic: 'foo';", ctx.toString());
|
||||
ctx = new SplittableMessageContext("foo", null, "blah".getBytes(StandardCharsets.UTF_8));
|
||||
assertEquals("topic: 'foo'; delimiter: 'blah'", ctx.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateNoNPEandNoSideffectsOnSetsGets() {
|
||||
SplittableMessageContext ctx = new SplittableMessageContext("foo", null, null);
|
||||
ctx.setFailedSegments(null);
|
||||
assertNull(ctx.getFailedSegments());
|
||||
|
||||
ctx.setFailedSegmentsAsByteArray(null);
|
||||
assertNull(ctx.getFailedSegments());
|
||||
|
||||
assertNull(ctx.getDelimiterBytes());
|
||||
assertNull(ctx.getKeyBytes());
|
||||
assertNull(ctx.getKeyBytesAsString());
|
||||
assertEquals("foo", ctx.getTopicName());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-bundle</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-kafka-pubsub-nar</artifactId>
|
||||
<packaging>nar</packaging>
|
||||
<description>NiFi NAR for interacting with Apache Kafka</description>
|
||||
<properties>
|
||||
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||
<source.skip>true</source.skip>
|
||||
</properties>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-pubsub-processors</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -0,0 +1,299 @@
|
|||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed 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.
|
||||
|
||||
APACHE NIFI SUBCOMPONENTS:
|
||||
|
||||
The Apache NiFi project contains subcomponents with separate copyright
|
||||
notices and license terms. Your use of the source code for the these
|
||||
subcomponents is subject to the terms and conditions of the following
|
||||
licenses.
|
||||
|
||||
The binary distribution of this product bundles 'Scala Library' under a BSD
|
||||
style license.
|
||||
|
||||
Copyright (c) 2002-2015 EPFL
|
||||
Copyright (c) 2011-2015 Typesafe, Inc.
|
||||
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without modification,
|
||||
are permitted provided that the following conditions are met:
|
||||
|
||||
Redistributions of source code must retain the above copyright notice, this list of
|
||||
conditions and the following disclaimer.
|
||||
|
||||
Redistributions in binary form must reproduce the above copyright notice, this list of
|
||||
conditions and the following disclaimer in the documentation and/or other materials
|
||||
provided with the distribution.
|
||||
|
||||
Neither the name of the EPFL nor the names of its contributors may be used to endorse
|
||||
or promote products derived from this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS
|
||||
OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
|
||||
AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
|
||||
CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
|
||||
DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER
|
||||
IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
|
||||
OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
The binary distribution of this product bundles 'JLine' under a BSD
|
||||
style license.
|
||||
|
||||
Copyright (c) 2002-2006, Marc Prud'hommeaux <mwp1@cornell.edu>
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or
|
||||
without modification, are permitted provided that the following
|
||||
conditions are met:
|
||||
|
||||
Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
|
||||
Redistributions in binary form must reproduce the above copyright
|
||||
notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with
|
||||
the distribution.
|
||||
|
||||
Neither the name of JLine nor the names of its contributors
|
||||
may be used to endorse or promote products derived from this
|
||||
software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
|
||||
BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
|
||||
AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
|
||||
EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
|
||||
FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
|
||||
OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
|
||||
PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
|
||||
AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
|
||||
LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
|
||||
IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
|
||||
OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
The binary distribution of this product bundles 'JOpt Simple' under an MIT
|
||||
style license.
|
||||
|
||||
Copyright (c) 2009 Paul R. Holser, Jr.
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining
|
||||
a copy of this software and associated documentation files (the
|
||||
"Software"), to deal in the Software without restriction, including
|
||||
without limitation the rights to use, copy, modify, merge, publish,
|
||||
distribute, sublicense, and/or sell copies of the Software, and to
|
||||
permit persons to whom the Software is furnished to do so, subject to
|
||||
the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be
|
||||
included in all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
|
||||
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
|
||||
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
|
||||
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
|
||||
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
|
@ -0,0 +1,72 @@
|
|||
nifi-kafka-nar
|
||||
Copyright 2014-2016 The Apache Software Foundation
|
||||
|
||||
This product includes software developed at
|
||||
The Apache Software Foundation (http://www.apache.org/).
|
||||
|
||||
******************
|
||||
Apache Software License v2
|
||||
******************
|
||||
|
||||
The following binary components are provided under the Apache Software License v2
|
||||
|
||||
(ASLv2) Apache Commons Lang
|
||||
The following NOTICE information applies:
|
||||
Apache Commons Lang
|
||||
Copyright 2001-2014 The Apache Software Foundation
|
||||
|
||||
This product includes software from the Spring Framework,
|
||||
under the Apache License 2.0 (see: StringUtils.containsWhitespace())
|
||||
|
||||
(ASLv2) Apache Kafka
|
||||
The following NOTICE information applies:
|
||||
Apache Kafka
|
||||
Copyright 2012 The Apache Software Foundation.
|
||||
|
||||
(ASLv2) Yammer Metrics
|
||||
The following NOTICE information applies:
|
||||
Metrics
|
||||
Copyright 2010-2012 Coda Hale and Yammer, Inc.
|
||||
|
||||
This product includes software developed by Coda Hale and Yammer, Inc.
|
||||
|
||||
This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
|
||||
with the following comments:
|
||||
|
||||
Written by Doug Lea with assistance from members of JCP JSR-166
|
||||
Expert Group and released to the public domain, as explained at
|
||||
http://creativecommons.org/publicdomain/zero/1.0/
|
||||
|
||||
(ASLv2) Snappy Java
|
||||
The following NOTICE information applies:
|
||||
This product includes software developed by Google
|
||||
Snappy: http://code.google.com/p/snappy/ (New BSD License)
|
||||
|
||||
This product includes software developed by Apache
|
||||
PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
|
||||
(Apache 2.0 license)
|
||||
|
||||
This library containd statically linked libstdc++. This inclusion is allowed by
|
||||
"GCC RUntime Library Exception"
|
||||
http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
|
||||
|
||||
(ASLv2) Apache ZooKeeper
|
||||
The following NOTICE information applies:
|
||||
Apache ZooKeeper
|
||||
Copyright 2009-2012 The Apache Software Foundation
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.1
|
||||
************************
|
||||
|
||||
The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
|
||||
|
||||
(CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
|
||||
|
||||
************************
|
||||
Common Development and Distribution License 1.0
|
||||
************************
|
||||
|
||||
The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details.
|
||||
|
||||
(CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
|
|
@ -0,0 +1,79 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-bundle</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<artifactId>nifi-kafka-pubsub-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka-clients</artifactId>
|
||||
<version>0.9.0.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka_2.10</artifactId>
|
||||
<version>0.9.0.1</version>
|
||||
<exclusions>
|
||||
<!-- Transitive dependencies excluded because they are located
|
||||
in a legacy Maven repository, which Maven 3 doesn't support. -->
|
||||
<exclusion>
|
||||
<groupId>javax.jms</groupId>
|
||||
<artifactId>jms</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.sun.jdmk</groupId>
|
||||
<artifactId>jmxtools</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.sun.jmx</groupId>
|
||||
<artifactId>jmxri</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-simple</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -0,0 +1,334 @@
|
|||
/*
|
||||
* 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.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||
import org.apache.kafka.clients.producer.ProducerConfig;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyDescriptor.Builder;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Base class for implementing {@link Processor}s to publish and consume
|
||||
* messages to/from Kafka
|
||||
*
|
||||
* @see PublishKafka
|
||||
* @see ConsumeKafka
|
||||
*/
|
||||
abstract class AbstractKafkaProcessor<T extends Closeable> extends AbstractSessionFactoryProcessor {
|
||||
|
||||
final Logger logger = LoggerFactory.getLogger(this.getClass());
|
||||
|
||||
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
|
||||
|
||||
private static final String BROKER_REGEX = SINGLE_BROKER_REGEX + "(?:,\\s*" + SINGLE_BROKER_REGEX + ")*";
|
||||
|
||||
|
||||
static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
|
||||
static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
|
||||
static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
|
||||
static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
|
||||
|
||||
static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
|
||||
.name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
|
||||
.displayName("Kafka Brokers")
|
||||
.description("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX)))
|
||||
.expressionLanguageSupported(true)
|
||||
.defaultValue("localhost:9092")
|
||||
.build();
|
||||
static final PropertyDescriptor CLIENT_ID = new PropertyDescriptor.Builder()
|
||||
.name(ProducerConfig.CLIENT_ID_CONFIG)
|
||||
.displayName("Client ID")
|
||||
.description("String value uniquely identifying this client application. Corresponds to Kafka's 'client.id' property.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
.build();
|
||||
static final PropertyDescriptor SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
|
||||
.name("security.protocol")
|
||||
.displayName("Security Protocol")
|
||||
.description("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
|
||||
.required(false)
|
||||
.expressionLanguageSupported(false)
|
||||
.allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
|
||||
.defaultValue(SEC_PLAINTEXT.getValue())
|
||||
.build();
|
||||
static final PropertyDescriptor KERBEROS_PRINCIPLE = new PropertyDescriptor.Builder()
|
||||
.name("sasl.kerberos.service.name")
|
||||
.displayName("Kerberos Service Name")
|
||||
.description("The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config. "
|
||||
+ "Corresponds to Kafka's 'security.protocol' property."
|
||||
+ "It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor TOPIC = new PropertyDescriptor.Builder()
|
||||
.name("topic")
|
||||
.displayName("Topic Name")
|
||||
.description("The name of the Kafka Topic")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
.build();
|
||||
|
||||
static final Builder MESSAGE_DEMARCATOR_BUILDER = new PropertyDescriptor.Builder()
|
||||
.name("message-demarcator")
|
||||
.displayName("Message Demarcator")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(true);
|
||||
|
||||
static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("All FlowFiles that are the are successfully sent to or received from Kafka are routed to this relationship")
|
||||
.build();
|
||||
|
||||
static final List<PropertyDescriptor> SHARED_DESCRIPTORS = new ArrayList<>();
|
||||
|
||||
static final Set<Relationship> SHARED_RELATIONSHIPS = new HashSet<>();
|
||||
|
||||
private final AtomicInteger taskCounter = new AtomicInteger();
|
||||
|
||||
private volatile boolean acceptTask = true;
|
||||
|
||||
static {
|
||||
SHARED_DESCRIPTORS.add(BOOTSTRAP_SERVERS);
|
||||
SHARED_DESCRIPTORS.add(TOPIC);
|
||||
SHARED_DESCRIPTORS.add(CLIENT_ID);
|
||||
SHARED_DESCRIPTORS.add(SECURITY_PROTOCOL);
|
||||
SHARED_DESCRIPTORS.add(KERBEROS_PRINCIPLE);
|
||||
SHARED_RELATIONSHIPS.add(REL_SUCCESS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Instance of {@link KafkaPublisher} or {@link KafkaConsumer}
|
||||
*/
|
||||
volatile T kafkaResource;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
@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 'kafkaResource' 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 'kafkaResource' can become null right after its null
|
||||
* check passed causing subsequent NPE.
|
||||
*/
|
||||
synchronized (this) {
|
||||
if (this.kafkaResource == null) {
|
||||
this.kafkaResource = this.buildKafkaResource(context, session);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* 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
|
||||
* - PublishKafka - some messages were sent to Kafka based on existence of the input FlowFile
|
||||
*/
|
||||
boolean processed = this.rendezvousWithKafka(context, session);
|
||||
session.commit();
|
||||
if (processed) {
|
||||
this.postCommit(context);
|
||||
} else {
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.kafkaResource != null) {
|
||||
try {
|
||||
this.kafkaResource.close();
|
||||
} catch (Exception e) {
|
||||
this.getLogger().warn("Failed while closing " + this.kafkaResource, e);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
this.kafkaResource = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
||||
return new PropertyDescriptor.Builder()
|
||||
.description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
|
||||
.name(propertyDescriptorName).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* This operation is called from
|
||||
* {@link #onTrigger(ProcessContext, ProcessSessionFactory)} method and
|
||||
* contains main processing logic for this Processor.
|
||||
*/
|
||||
protected abstract boolean rendezvousWithKafka(ProcessContext context, ProcessSession session);
|
||||
|
||||
/**
|
||||
* Builds target resource for interacting with Kafka. The target resource
|
||||
* could be one of {@link KafkaPublisher} or {@link KafkaConsumer}
|
||||
*/
|
||||
protected abstract T buildKafkaResource(ProcessContext context, ProcessSession session);
|
||||
|
||||
/**
|
||||
* This operation will be executed after {@link ProcessSession#commit()} has
|
||||
* been called.
|
||||
*/
|
||||
protected void postCommit(ProcessContext context) {
|
||||
// no op
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
|
||||
List<ValidationResult> results = new ArrayList<>();
|
||||
|
||||
String securityProtocol = validationContext.getProperty(SECURITY_PROTOCOL).getValue();
|
||||
|
||||
/*
|
||||
* validates that if one of SASL (Kerberos) option is selected for
|
||||
* security protocol, then Kerberos principal is provided as well
|
||||
*/
|
||||
if (SEC_SASL_PLAINTEXT.getValue().equals(securityProtocol) || SEC_SASL_SSL.getValue().equals(securityProtocol)){
|
||||
String kerberosPrincipal = validationContext.getProperty(KERBEROS_PRINCIPLE).getValue();
|
||||
if (kerberosPrincipal == null || kerberosPrincipal.trim().length() == 0){
|
||||
results.add(new ValidationResult.Builder().subject(KERBEROS_PRINCIPLE.getDisplayName()).valid(false)
|
||||
.explanation("The <" + KERBEROS_PRINCIPLE.getDisplayName() + "> property must be set when <"
|
||||
+ SECURITY_PROTOCOL.getDisplayName() + "> is configured as '"
|
||||
+ SEC_SASL_PLAINTEXT.getValue() + "' or '" + SEC_SASL_SSL.getValue() + "'.")
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds transit URI for provenance event. The transit URI will be in the
|
||||
* form of <security.protocol>://<bootstrap.servers>/topic
|
||||
*/
|
||||
String buildTransitURI(String securityProtocol, String brokers, String topic) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(securityProtocol);
|
||||
builder.append("://");
|
||||
builder.append(brokers);
|
||||
builder.append("/");
|
||||
builder.append(topic);
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds Kafka {@link Properties}
|
||||
*/
|
||||
Properties buildKafkaProperties(ProcessContext context) {
|
||||
Properties properties = new Properties();
|
||||
for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) {
|
||||
String pName = propertyDescriptor.getName();
|
||||
String pValue = 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));
|
||||
}
|
||||
properties.setProperty(pName, pValue);
|
||||
}
|
||||
}
|
||||
return properties;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,242 @@
|
|||
/*
|
||||
* 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.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
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.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Consumes messages from Apache Kafka")
|
||||
@Tags({ "Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume" })
|
||||
public class ConsumeKafka extends AbstractKafkaProcessor<Consumer<byte[], byte[]>> {
|
||||
|
||||
static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
|
||||
|
||||
static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
|
||||
|
||||
static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group");
|
||||
|
||||
static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
|
||||
.name(ConsumerConfig.GROUP_ID_CONFIG)
|
||||
.displayName("Group ID")
|
||||
.description("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.build();
|
||||
static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
|
||||
.name(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)
|
||||
.displayName("Offset Reset")
|
||||
.description("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any "
|
||||
+ "more on the server (e.g. because that data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.")
|
||||
.required(true)
|
||||
.allowableValues(OFFSET_EARLIEST, OFFSET_LATEST, OFFSET_NONE)
|
||||
.defaultValue(OFFSET_LATEST.getValue())
|
||||
.build();
|
||||
static final PropertyDescriptor MESSAGE_DEMARCATOR = MESSAGE_DEMARCATOR_BUILDER
|
||||
.description("Since KafkaConsumer receives messages in batches, you have an option to output a single FlowFile which contains "
|
||||
+ "all Kafka messages in a single batch and this property allows you to provide a string (interpreted as UTF-8) to use "
|
||||
+ "for demarcating apart multiple Kafka messages. This is an optional property and if not provided each Kafka message received "
|
||||
+ "in a batch will result in a single FlowFile which essentially means that this processor may output multiple FlowFiles for each "
|
||||
+ "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS")
|
||||
.build();
|
||||
|
||||
|
||||
static final List<PropertyDescriptor> DESCRIPTORS;
|
||||
|
||||
static final Set<Relationship> RELATIONSHIPS;
|
||||
|
||||
private volatile byte[] demarcatorBytes;
|
||||
|
||||
private volatile String topic;
|
||||
|
||||
private volatile String brokers;
|
||||
|
||||
/*
|
||||
* Will ensure that the list of the PropertyDescriptors is build only once,
|
||||
* since all other lifecycle methods are invoked multiple times.
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _descriptors = new ArrayList<>();
|
||||
_descriptors.addAll(SHARED_DESCRIPTORS);
|
||||
_descriptors.add(GROUP_ID);
|
||||
_descriptors.add(AUTO_OFFSET_RESET);
|
||||
_descriptors.add(MESSAGE_DEMARCATOR);
|
||||
DESCRIPTORS = Collections.unmodifiableList(_descriptors);
|
||||
|
||||
RELATIONSHIPS = Collections.unmodifiableSet(SHARED_RELATIONSHIPS);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return RELATIONSHIPS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will unsubscribe form {@link KafkaConsumer} delegating to 'super' to do
|
||||
* the rest.
|
||||
*/
|
||||
@Override
|
||||
@OnStopped
|
||||
public void close() {
|
||||
if (this.kafkaResource != null) {
|
||||
try {
|
||||
this.kafkaResource.unsubscribe();
|
||||
} finally { // in the event the above fails
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return DESCRIPTORS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will rendezvous with Kafka by performing the following:
|
||||
* <br>
|
||||
* - poll {@link ConsumerRecords} from {@link KafkaConsumer} in a
|
||||
* non-blocking manner, signaling yield if no records were received from
|
||||
* Kafka
|
||||
* <br>
|
||||
* - if records were received form Kafka, the are written to a newly created
|
||||
* {@link FlowFile}'s {@link OutputStream} using a provided demarcator (see
|
||||
* {@link #MESSAGE_DEMARCATOR}
|
||||
*/
|
||||
@Override
|
||||
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession processSession) {
|
||||
ConsumerRecords<byte[], byte[]> consumedRecords = this.kafkaResource.poll(100);
|
||||
if (consumedRecords != null && !consumedRecords.isEmpty()) {
|
||||
long start = System.nanoTime();
|
||||
FlowFile flowFile = processSession.create();
|
||||
final AtomicInteger messageCounter = new AtomicInteger();
|
||||
|
||||
final Iterator<ConsumerRecord<byte[], byte[]>> iter = consumedRecords.iterator();
|
||||
while (iter.hasNext()){
|
||||
flowFile = processSession.append(flowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
ConsumerRecord<byte[], byte[]> consumedRecord = iter.next();
|
||||
if (messageCounter.getAndIncrement() > 0 && ConsumeKafka.this.demarcatorBytes != null) {
|
||||
out.write(ConsumeKafka.this.demarcatorBytes);
|
||||
}
|
||||
out.write(consumedRecord.value());
|
||||
}
|
||||
});
|
||||
/*
|
||||
* Release FlowFile if there are more messages in the
|
||||
* ConsumerRecords batch and no demarcator was provided,
|
||||
* otherwise the FlowFile will be released as soon as this loop
|
||||
* exits.
|
||||
*/
|
||||
if (iter.hasNext() && ConsumeKafka.this.demarcatorBytes == null){
|
||||
this.releaseFlowFile(flowFile, context, processSession, start, messageCounter.get());
|
||||
flowFile = processSession.create();
|
||||
messageCounter.set(0);
|
||||
}
|
||||
}
|
||||
this.releaseFlowFile(flowFile, context, processSession, start, messageCounter.get());
|
||||
}
|
||||
return consumedRecords != null && !consumedRecords.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* This operation is called from
|
||||
* {@link #onTrigger(ProcessContext, ProcessSessionFactory)} method after
|
||||
* the process session is committed so that then kafka offset changes can be
|
||||
* committed. This can mean in cases of really bad timing we could have data
|
||||
* duplication upon recovery but not data loss. We want to commit the flow
|
||||
* files in a NiFi sense before we commit them in a Kafka sense.
|
||||
*/
|
||||
@Override
|
||||
protected void postCommit(ProcessContext context) {
|
||||
this.kafkaResource.commitSync();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds and instance of {@link KafkaConsumer} and subscribes to a provided
|
||||
* topic.
|
||||
*/
|
||||
@Override
|
||||
protected Consumer<byte[], byte[]> buildKafkaResource(ProcessContext context, ProcessSession session) {
|
||||
this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet()
|
||||
? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
|
||||
: null;
|
||||
this.topic = context.getProperty(TOPIC).evaluateAttributeExpressions().getValue();
|
||||
this.brokers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
|
||||
|
||||
Properties kafkaProperties = this.buildKafkaProperties(context);
|
||||
kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
|
||||
kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
|
||||
|
||||
KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(kafkaProperties);
|
||||
consumer.subscribe(Collections.singletonList(this.topic));
|
||||
return consumer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will release flow file. Releasing of the flow file in the context of this
|
||||
* operation implies the following:
|
||||
*
|
||||
* If Empty then remove from session and return If has something then
|
||||
* transfer to {@link #REL_SUCCESS}
|
||||
*/
|
||||
private void releaseFlowFile(FlowFile flowFile, ProcessContext context, ProcessSession session, long start, int msgCount) {
|
||||
long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
||||
String transitUri = this.buildTransitURI(context.getProperty(SECURITY_PROTOCOL).getValue(), this.brokers, topic);
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri, "Received " + msgCount + " Kafka messages", executionDuration);
|
||||
this.getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[] { flowFile, msgCount, executionDuration });
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,232 @@
|
|||
/*
|
||||
* 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.ProcessorLog;
|
||||
import org.apache.nifi.stream.io.util.StreamDemarcator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Wrapper over {@link KafkaProducer} to assist {@link PublishKafka} processor
|
||||
* with sending contents of the {@link FlowFile}s to Kafka.
|
||||
*/
|
||||
class KafkaPublisher implements Closeable {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(KafkaPublisher.class);
|
||||
|
||||
private final Producer<byte[], byte[]> kafkaProducer;
|
||||
|
||||
private volatile long ackWaitTime = 30000;
|
||||
|
||||
private volatile ProcessorLog processLog;
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
this.kafkaProducer = new KafkaProducer<>(kafkaProperties);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
for (; (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));
|
||||
}
|
||||
}
|
||||
|
||||
int lastAckedMessageIndex = this.processAcks(resultFutures, prevLastAckedMessageIndex);
|
||||
return new KafkaPublisherResult(tokenCounter, lastAckedMessageIndex);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Will set {@link ProcessorLog} as an additional logger to forward log
|
||||
* messages to NiFi bulletin
|
||||
*/
|
||||
void setProcessLog(ProcessorLog processLog) {
|
||||
this.processLog = processLog;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private void warnOrError(String message, Exception e) {
|
||||
if (e == null) {
|
||||
logger.warn(message);
|
||||
if (this.processLog != null) {
|
||||
this.processLog.warn(message);
|
||||
}
|
||||
} else {
|
||||
logger.error(message, e);
|
||||
if (this.processLog != null) {
|
||||
this.processLog.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.messagesSent - 1 == this.lastMessageAcked;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Sent:" + this.messagesSent + "; Last ACK:" + this.lastMessageAcked;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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.Map;
|
||||
|
||||
import org.apache.kafka.clients.producer.Partitioner;
|
||||
import org.apache.kafka.common.Cluster;
|
||||
|
||||
/**
|
||||
* Collection of implementation of common Kafka {@link Partitioner}s.
|
||||
*/
|
||||
final public class Partitioners {
|
||||
|
||||
private Partitioners() {
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Partitioner} that implements 'round-robin' mechanism which evenly
|
||||
* distributes load between all available partitions.
|
||||
*/
|
||||
public static class RoundRobinPartitioner implements Partitioner {
|
||||
private volatile int index;
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> configs) {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) {
|
||||
return this.next(cluster.availablePartitionsForTopic(topic).size());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// noop
|
||||
}
|
||||
|
||||
private synchronized int next(int numberOfPartitions) {
|
||||
if (this.index >= numberOfPartitions) {
|
||||
this.index = 0;
|
||||
}
|
||||
return index++;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,359 @@
|
|||
/*
|
||||
* 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.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
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.AtomicReference;
|
||||
|
||||
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.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.kafka.pubsub.KafkaPublisher.KafkaPublisherResult;
|
||||
import org.apache.nifi.processors.kafka.pubsub.Partitioners.RoundRobinPartitioner;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" })
|
||||
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka. The messages to send may be individual FlowFiles or may be delimited, using a "
|
||||
+ "user-specified delimiter, such as a new-line.")
|
||||
@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
|
||||
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 AbstractKafkaProcessor<KafkaPublisher> {
|
||||
|
||||
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";
|
||||
|
||||
protected static final String MSG_COUNT = "msg.count";
|
||||
|
||||
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
|
||||
"FlowFile will be routed to failure unless the message is replicated to the appropriate "
|
||||
+ "number of Kafka Nodes according to the Topic configuration");
|
||||
static final AllowableValue DELIVERY_ONE_NODE = new AllowableValue("1", "Guarantee Single Node Delivery",
|
||||
"FlowFile will be routed to success if the message is received by a single Kafka node, "
|
||||
+ "whether or not it is replicated. This is faster than <Guarantee Replicated Delivery> "
|
||||
+ "but can result in data loss if a Kafka node crashes");
|
||||
static final AllowableValue DELIVERY_BEST_EFFORT = new AllowableValue("0", "Best Effort",
|
||||
"FlowFile will be routed to success after successfully writing the content to a Kafka node, "
|
||||
+ "without waiting for a response. This provides the best performance but may result in data loss.");
|
||||
|
||||
static final AllowableValue ROUND_ROBIN_PARTITIONING = new AllowableValue(RoundRobinPartitioner.class.getName(),
|
||||
RoundRobinPartitioner.class.getSimpleName(),
|
||||
"Messages will be assigned partitions in a round-robin fashion, sending the first message to Partition 1, "
|
||||
+ "the next Partition to Partition 2, and so on, wrapping as necessary.");
|
||||
static final AllowableValue RANDOM_PARTITIONING = new AllowableValue("org.apache.kafka.clients.producer.internals.DefaultPartitioner",
|
||||
"DefaultPartitioner", "Messages will be assigned to random partitions.");
|
||||
|
||||
static final PropertyDescriptor DELIVERY_GUARANTEE = new PropertyDescriptor.Builder()
|
||||
.name(ProducerConfig.ACKS_CONFIG)
|
||||
.displayName("Delivery Guarantee")
|
||||
.description("Specifies the requirement for guaranteeing that a message is sent to Kafka. Corresponds to Kafka's 'acks' property.")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(false)
|
||||
.allowableValues(DELIVERY_BEST_EFFORT, DELIVERY_ONE_NODE, DELIVERY_REPLICATED)
|
||||
.defaultValue(DELIVERY_BEST_EFFORT.getValue())
|
||||
.build();
|
||||
static final PropertyDescriptor META_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 "
|
||||
+ "entire 'send' call. Corresponds to Kafka's 'max.block.ms' property")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
.defaultValue("30 sec")
|
||||
.build();
|
||||
static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
|
||||
.name("kafka-key")
|
||||
.displayName("Kafka Key")
|
||||
.description("The Key to use for the Message")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
.build();
|
||||
static final PropertyDescriptor MESSAGE_DEMARCATOR = MESSAGE_DEMARCATOR_BUILDER
|
||||
.description("Specifies the string (interpreted as UTF-8) to use for demarcating apart multiple messages within "
|
||||
+ "a single FlowFile. If not specified, the entire content of the FlowFile will be used as a single message. If specified, the "
|
||||
+ "contents of the FlowFile will be split on this delimiter and each section sent as a separate Kafka message. "
|
||||
+ "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()
|
||||
.name(ProducerConfig.PARTITIONER_CLASS_CONFIG)
|
||||
.displayName("Partitioner class")
|
||||
.description("Specifies which class to use to compute a partition id for a message. Corresponds to Kafka's 'partitioner.class' property.")
|
||||
.allowableValues(ROUND_ROBIN_PARTITIONING, RANDOM_PARTITIONING)
|
||||
.defaultValue(RANDOM_PARTITIONING.getValue())
|
||||
.required(false)
|
||||
.build();
|
||||
static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder()
|
||||
.name(ProducerConfig.COMPRESSION_TYPE_CONFIG)
|
||||
.displayName("Compression Type")
|
||||
.description("This parameter allows you to specify the compression codec for all data generated by this producer.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.allowableValues("none", "gzip", "snappy", "lz4")
|
||||
.defaultValue("none")
|
||||
.build();
|
||||
|
||||
static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("Any FlowFile that cannot be sent to Kafka will be routed to this Relationship")
|
||||
.build();
|
||||
|
||||
static final List<PropertyDescriptor> DESCRIPTORS;
|
||||
|
||||
static final Set<Relationship> RELATIONSHIPS;
|
||||
|
||||
private volatile String brokers;
|
||||
|
||||
/*
|
||||
* Will ensure that list of PropertyDescriptors is build only once, since
|
||||
* all other lifecycle methods are invoked multiple times.
|
||||
*/
|
||||
static {
|
||||
List<PropertyDescriptor> _descriptors = new ArrayList<>();
|
||||
_descriptors.addAll(SHARED_DESCRIPTORS);
|
||||
_descriptors.add(DELIVERY_GUARANTEE);
|
||||
_descriptors.add(KEY);
|
||||
_descriptors.add(MESSAGE_DEMARCATOR);
|
||||
_descriptors.add(META_WAIT_TIME);
|
||||
_descriptors.add(PARTITION_CLASS);
|
||||
_descriptors.add(COMPRESSION_CODEC);
|
||||
|
||||
DESCRIPTORS = Collections.unmodifiableList(_descriptors);
|
||||
|
||||
Set<Relationship> _relationships = new HashSet<>();
|
||||
_relationships.addAll(SHARED_RELATIONSHIPS);
|
||||
_relationships.add(REL_FAILURE);
|
||||
RELATIONSHIPS = Collections.unmodifiableSet(_relationships);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return RELATIONSHIPS;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return DESCRIPTORS;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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}
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
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 = this.buildTransitURI(context.getProperty(SECURITY_PROTOCOL).getValue(), this.brokers, topic);
|
||||
session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration);
|
||||
this.getLogger().info("Successfully sent {} to Kafka as {} message(s) in {} millis", new Object[] { flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration });
|
||||
} else {
|
||||
relationship = REL_FAILURE;
|
||||
flowFile = session.penalize(flowFile);
|
||||
}
|
||||
session.transfer(flowFile, relationship);
|
||||
}
|
||||
return flowFile != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds and instance of {@link KafkaPublisher}.
|
||||
*/
|
||||
@Override
|
||||
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session) {
|
||||
Properties kafkaProperties = this.buildKafkaProperties(context);
|
||||
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
|
||||
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
|
||||
this.brokers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue();
|
||||
return new KafkaPublisher(kafkaProperties);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<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);
|
||||
KafkaPublisherResult result = PublishKafka.this.kafkaResource.publish(publishingContext);
|
||||
publishResultRef.set(result);
|
||||
}
|
||||
});
|
||||
|
||||
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 PublishKafka processor it is treated as a fresh FlowFile
|
||||
* regardless if it has #FAILED* attributes set.
|
||||
*/
|
||||
private PublishingContext buildPublishingContext(FlowFile flowFile, ProcessContext context, InputStream contentStream) {
|
||||
String topicName;
|
||||
byte[] keyBytes;
|
||||
byte[] delimiterBytes = null;
|
||||
int lastAckedMessageIndex = -1;
|
||||
if (this.isFailedFlowFile(flowFile)) {
|
||||
lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX));
|
||||
topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);
|
||||
keyBytes = flowFile.getAttribute(FAILED_KEY_ATTR) != null
|
||||
? flowFile.getAttribute(FAILED_KEY_ATTR).getBytes(StandardCharsets.UTF_8) : null;
|
||||
delimiterBytes = flowFile.getAttribute(FAILED_DELIMITER_ATTR) != null
|
||||
? flowFile.getAttribute(FAILED_DELIMITER_ATTR).getBytes(StandardCharsets.UTF_8) : null;
|
||||
|
||||
} else {
|
||||
topicName = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue();
|
||||
String _key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||
keyBytes = _key == null ? null : _key.getBytes(StandardCharsets.UTF_8);
|
||||
delimiterBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR)
|
||||
.evaluateAttributeExpressions(flowFile).getValue().getBytes(StandardCharsets.UTF_8) : null;
|
||||
}
|
||||
|
||||
PublishingContext publishingContext = new PublishingContext(contentStream, topicName, lastAckedMessageIndex);
|
||||
publishingContext.setKeyBytes(keyBytes);
|
||||
publishingContext.setDelimiterBytes(delimiterBytes);
|
||||
return publishingContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,139 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/*
|
||||
* We're using the default value from Kafka. We are using it to control the
|
||||
* message size before it goes to to Kafka thus limiting possibility of a
|
||||
* late failures in Kafka client.
|
||||
*/
|
||||
private int maxRequestSize = 1048576; // kafka default
|
||||
|
||||
private boolean maxRequestSizeSet;
|
||||
|
||||
private byte[] keyBytes;
|
||||
|
||||
private byte[] delimiterBytes;
|
||||
|
||||
PublishingContext(InputStream contentStream, String topic) {
|
||||
this(contentStream, topic, -1);
|
||||
}
|
||||
|
||||
PublishingContext(InputStream contentStream, String topic, int lastAckedMessageIndex) {
|
||||
this.validateInput(contentStream, topic, lastAckedMessageIndex);
|
||||
this.contentStream = contentStream;
|
||||
this.topic = topic;
|
||||
this.lastAckedMessageIndex = lastAckedMessageIndex;
|
||||
}
|
||||
|
||||
@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");
|
||||
}
|
||||
}
|
||||
|
||||
void setMaxRequestSize(int maxRequestSize) {
|
||||
if (!this.maxRequestSizeSet) {
|
||||
if (maxRequestSize > 0) {
|
||||
this.maxRequestSize = maxRequestSize;
|
||||
this.maxRequestSizeSet = true;
|
||||
} else {
|
||||
throw new IllegalArgumentException("'maxRequestSize' must be > 0");
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("'maxRequestSize' 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");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
# 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.
|
||||
org.apache.nifi.processors.kafka.pubsub.PublishKafka
|
||||
org.apache.nifi.processors.kafka.pubsub.ConsumeKafka
|
|
@ -0,0 +1,33 @@
|
|||
<!DOCTYPE html>
|
||||
<html lang="en">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<head>
|
||||
<meta charset="utf-8" />
|
||||
<title>ConsumeKafka</title>
|
||||
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
||||
</head>
|
||||
|
||||
<body>
|
||||
<!-- Processor Documentation ================================================== -->
|
||||
<h2>Description:</h2>
|
||||
<p>
|
||||
This Processors polls <a href="http://kafka.apache.org/">Apache Kafka</a>
|
||||
for data using KafkaConsumer API available with Kafka 0.9+. When a message is received
|
||||
from Kafka, this Processor emits a FlowFile where the content of the FlowFile is the value
|
||||
of the Kafka message.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
|
@ -0,0 +1,47 @@
|
|||
<!DOCTYPE html>
|
||||
<html lang="en">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<head>
|
||||
<meta charset="utf-8" />
|
||||
<title>PublishKafka</title>
|
||||
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
||||
</head>
|
||||
|
||||
<body>
|
||||
<!-- Processor Documentation ================================================== -->
|
||||
<h2>Description:</h2>
|
||||
<p>
|
||||
This Processors puts the contents of a FlowFile to a Topic in
|
||||
<a href="http://kafka.apache.org/">Apache Kafka</a> using KafkaProducer API available
|
||||
with Kafka 0.9+ API. The content of a FlowFile becomes the contents of a Kafka message.
|
||||
This message is optionally assigned a key by using the <Kafka Key> Property.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
The Processor allows the user to configure an optional Message Demarcator that
|
||||
can be used to send many messages per FlowFile. For example, a <i>\n</i> could be used
|
||||
to indicate that the contents of the FlowFile should be used to send one message
|
||||
per line of text. It also supports multi-char demarcators (e.g., 'my custom demarcator').
|
||||
If the property is not set, the entire contents of the FlowFile
|
||||
will be sent as a single message. When using the demarcator, if some messages are
|
||||
successfully sent but other messages fail to send, the resulting FlowFile will be
|
||||
considered a failed FlowFuile and will have additional attributes to that effect.
|
||||
One of such attributes is 'failed.last.idx' which indicates the index of the last message
|
||||
that was successfully ACKed by Kafka. (if no demarcator is used the value of this index will be -1).
|
||||
This will allow PublishKafka to only re-send un-ACKed messages on the next re-try.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
|
@ -0,0 +1,456 @@
|
|||
/*
|
||||
* 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.doAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
public class AbstractKafkaProcessorLifecycelTest {
|
||||
|
||||
private final static Random random = new Random();
|
||||
|
||||
@Test
|
||||
public void validateBaseProperties() throws Exception {
|
||||
TestRunner runner = TestRunners.newTestRunner(DummyProcessor.class);
|
||||
runner.setProperty(AbstractKafkaProcessor.BOOTSTRAP_SERVERS, "");
|
||||
runner.setProperty(AbstractKafkaProcessor.TOPIC, "foo");
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "foo");
|
||||
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "foo");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("'bootstrap.servers' validated against 'foo' is invalid"));
|
||||
}
|
||||
runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "foo:1234");
|
||||
|
||||
runner.removeProperty(ConsumeKafka.TOPIC);
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("'topic' is invalid because topic is required"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.TOPIC, "");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.TOPIC, " ");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
runner.setProperty(ConsumeKafka.TOPIC, "blah");
|
||||
|
||||
runner.removeProperty(ConsumeKafka.CLIENT_ID);
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("invalid because client.id is required"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, " ");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "ghj");
|
||||
|
||||
runner.setProperty(PublishKafka.KERBEROS_PRINCIPLE, "");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
runner.setProperty(PublishKafka.KERBEROS_PRINCIPLE, " ");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore // just for extra sanity check
|
||||
public void validateConcurrencyWithRandomFailuresMultiple() throws Exception {
|
||||
for (int i = 0; i < 100; i++) {
|
||||
validateConcurrencyWithRandomFailures();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateConcurrencyWithRandomFailures() throws Exception {
|
||||
ExecutorService processingExecutor = Executors.newFixedThreadPool(32);
|
||||
final AtomicInteger commitCounter = new AtomicInteger();
|
||||
final AtomicInteger rollbackCounter = new AtomicInteger();
|
||||
final AtomicInteger yieldCounter = new AtomicInteger();
|
||||
|
||||
final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class);
|
||||
final ProcessSession session = mock(ProcessSession.class);
|
||||
when(sessionFactory.createSession()).thenReturn(session);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
commitCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).commit();
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
rollbackCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).rollback(true);
|
||||
|
||||
final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor());
|
||||
|
||||
int testCount = 1000;
|
||||
final CountDownLatch latch = new CountDownLatch(testCount);
|
||||
for (int i = 0; i < testCount; i++) {
|
||||
processingExecutor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
ProcessContext context = mock(ProcessContext.class);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
yieldCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(context).yield();
|
||||
if (random.nextInt(10) == 5) {
|
||||
when(context.getName()).thenReturn("fail");
|
||||
}
|
||||
try {
|
||||
processor.onTrigger(context, sessionFactory);
|
||||
} catch (Exception e) {
|
||||
fail();
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
assertTrue(latch.await(20000, TimeUnit.MILLISECONDS));
|
||||
processingExecutor.shutdown();
|
||||
|
||||
System.out.println("SUCCESS: " + processor.successfulTriggers);
|
||||
System.out.println("FAILURE: " + processor.failedTriggers);
|
||||
System.out.println("INIT: " + processor.resourceReinitialized);
|
||||
System.out.println("YIELD CALLS: " + yieldCounter.get());
|
||||
System.out.println("COMMIT CALLS: " + commitCounter.get());
|
||||
System.out.println("ROLLBACK CALLS: " + rollbackCounter.get());
|
||||
System.out.println("Close CALLS: " + processor.closeCounter.get());
|
||||
|
||||
/*
|
||||
* this has to be <= 1 since the last thread may come to finally block
|
||||
* after acceptTask flag has been reset at which point the close will
|
||||
* not be called (which is correct behavior since it will be invoked
|
||||
* explicitly by the life-cycle operations of a running processor).
|
||||
*
|
||||
* You can actually observe the =1 behavior in the next test where it is
|
||||
* always 0 close calls
|
||||
*/
|
||||
int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get();
|
||||
assertTrue(closeVsInitDiff <= 1);
|
||||
|
||||
assertEquals(commitCounter.get(), processor.successfulTriggers.get());
|
||||
assertEquals(rollbackCounter.get(), processor.failedTriggers.get());
|
||||
|
||||
assertEquals(testCount,
|
||||
processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateConcurrencyWithAllSuccesses() throws Exception {
|
||||
ExecutorService processingExecutor = Executors.newFixedThreadPool(32);
|
||||
final AtomicInteger commitCounter = new AtomicInteger();
|
||||
final AtomicInteger rollbackCounter = new AtomicInteger();
|
||||
final AtomicInteger yieldCounter = new AtomicInteger();
|
||||
|
||||
final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class);
|
||||
final ProcessSession session = mock(ProcessSession.class);
|
||||
when(sessionFactory.createSession()).thenReturn(session);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
commitCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).commit();
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
rollbackCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).rollback(true);
|
||||
|
||||
final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor());
|
||||
|
||||
int testCount = 1000;
|
||||
final CountDownLatch latch = new CountDownLatch(testCount);
|
||||
for (int i = 0; i < testCount; i++) {
|
||||
processingExecutor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
ProcessContext context = mock(ProcessContext.class);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
yieldCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(context).yield();
|
||||
try {
|
||||
processor.onTrigger(context, sessionFactory);
|
||||
} catch (Exception e) {
|
||||
fail();
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
assertTrue(latch.await(30000, TimeUnit.MILLISECONDS));
|
||||
processingExecutor.shutdown();
|
||||
|
||||
System.out.println("SUCCESS: " + processor.successfulTriggers);
|
||||
System.out.println("FAILURE: " + processor.failedTriggers);
|
||||
System.out.println("INIT: " + processor.resourceReinitialized);
|
||||
System.out.println("YIELD CALLS: " + yieldCounter.get());
|
||||
System.out.println("COMMIT CALLS: " + commitCounter.get());
|
||||
System.out.println("ROLLBACK CALLS: " + rollbackCounter.get());
|
||||
System.out.println("Close CALLS: " + processor.closeCounter.get());
|
||||
|
||||
/*
|
||||
* unlike previous test this one will always be 1 since there are no
|
||||
* failures
|
||||
*/
|
||||
int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get();
|
||||
assertEquals(1, closeVsInitDiff);
|
||||
|
||||
assertEquals(commitCounter.get(), processor.successfulTriggers.get());
|
||||
assertEquals(rollbackCounter.get(), processor.failedTriggers.get());
|
||||
|
||||
assertEquals(testCount,
|
||||
processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateConcurrencyWithAllFailures() throws Exception {
|
||||
ExecutorService processingExecutor = Executors.newFixedThreadPool(32);
|
||||
final AtomicInteger commitCounter = new AtomicInteger();
|
||||
final AtomicInteger rollbackCounter = new AtomicInteger();
|
||||
final AtomicInteger yieldCounter = new AtomicInteger();
|
||||
|
||||
final ProcessSessionFactory sessionFactory = mock(ProcessSessionFactory.class);
|
||||
final ProcessSession session = mock(ProcessSession.class);
|
||||
when(sessionFactory.createSession()).thenReturn(session);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
commitCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).commit();
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
rollbackCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(session).rollback(true);
|
||||
|
||||
final ConcurrencyValidatingProcessor processor = spy(new ConcurrencyValidatingProcessor());
|
||||
|
||||
int testCount = 1000;
|
||||
final CountDownLatch latch = new CountDownLatch(testCount);
|
||||
for (int i = 0; i < testCount; i++) {
|
||||
processingExecutor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
ProcessContext context = mock(ProcessContext.class);
|
||||
doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
yieldCounter.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
}).when(context).yield();
|
||||
when(context.getName()).thenReturn("fail");
|
||||
try {
|
||||
processor.onTrigger(context, sessionFactory);
|
||||
} catch (Exception e) {
|
||||
fail();
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
assertTrue(latch.await(20000, TimeUnit.MILLISECONDS));
|
||||
processingExecutor.shutdown();
|
||||
|
||||
System.out.println("SUCCESS: " + processor.successfulTriggers);
|
||||
System.out.println("FAILURE: " + processor.failedTriggers);
|
||||
System.out.println("INIT: " + processor.resourceReinitialized);
|
||||
System.out.println("YIELD CALLS: " + yieldCounter.get());
|
||||
System.out.println("COMMIT CALLS: " + commitCounter.get());
|
||||
System.out.println("ROLLBACK CALLS: " + rollbackCounter.get());
|
||||
System.out.println("Close CALLS: " + processor.closeCounter.get());
|
||||
|
||||
/*
|
||||
* unlike previous test this one will always be 0 since all triggers are
|
||||
* failures
|
||||
*/
|
||||
int closeVsInitDiff = processor.resourceReinitialized.get() - processor.closeCounter.get();
|
||||
assertEquals(0, closeVsInitDiff);
|
||||
|
||||
assertEquals(commitCounter.get(), processor.successfulTriggers.get());
|
||||
assertEquals(rollbackCounter.get(), processor.failedTriggers.get());
|
||||
|
||||
assertEquals(testCount,
|
||||
processor.successfulTriggers.get() + processor.failedTriggers.get() + yieldCounter.get());
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public static class DummyProcessor extends AbstractKafkaProcessor<Closeable> {
|
||||
@Override
|
||||
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Closeable buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
return mock(Closeable.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return SHARED_DESCRIPTORS;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static class ConcurrencyValidatingProcessor extends AbstractKafkaProcessor<Closeable> {
|
||||
final AtomicInteger failedTriggers = new AtomicInteger();
|
||||
final AtomicInteger successfulTriggers = new AtomicInteger();
|
||||
final AtomicInteger resourceReinitialized = new AtomicInteger();
|
||||
final AtomicInteger closeCounter = new AtomicInteger();
|
||||
|
||||
ConcurrencyValidatingProcessor() {
|
||||
try {
|
||||
Field loggerField = AbstractSessionFactoryProcessor.class.getDeclaredField("logger");
|
||||
loggerField.setAccessible(true);
|
||||
loggerField.set(this, mock(ProcessorLog.class));
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@OnStopped
|
||||
public void close() {
|
||||
super.close();
|
||||
assertTrue(this.kafkaResource == null);
|
||||
closeCounter.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session) {
|
||||
assertNotNull(this.kafkaResource);
|
||||
if ("fail".equals(context.getName())) {
|
||||
failedTriggers.incrementAndGet();
|
||||
throw new RuntimeException("Intentional");
|
||||
}
|
||||
this.successfulTriggers.incrementAndGet();
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Closeable buildKafkaResource(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
this.resourceReinitialized.incrementAndGet();
|
||||
return mock(Closeable.class);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,167 @@
|
|||
/*
|
||||
* 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.junit.Assert.fail;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Test;
|
||||
|
||||
// The test is valid and should be ran when working on this module. @Ignore is
|
||||
// to speed up the overall build
|
||||
public class ConsumeKafkaTest {
|
||||
|
||||
@Test
|
||||
public void validatePropertiesValidation() throws Exception {
|
||||
ConsumeKafka consumeKafka = new ConsumeKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(consumeKafka);
|
||||
runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "okeydokey:1234");
|
||||
runner.setProperty(ConsumeKafka.TOPIC, "foo");
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.GROUP_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
|
||||
|
||||
runner.removeProperty(ConsumeKafka.GROUP_ID);
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("invalid because group.id is required"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.GROUP_ID, "");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
|
||||
runner.setProperty(ConsumeKafka.GROUP_ID, " ");
|
||||
try {
|
||||
runner.assertValid();
|
||||
fail();
|
||||
} catch (AssertionError e) {
|
||||
assertTrue(e.getMessage().contains("must contain at least one character that is not white space"));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will set auto-offset to 'smallest' to ensure that all events (the once
|
||||
* that were sent before and after consumer startup) are received.
|
||||
*/
|
||||
@Test
|
||||
public void validateGetAllMessages() throws Exception {
|
||||
String topicName = "validateGetAllMessages";
|
||||
|
||||
StubConsumeKafka consumeKafka = new StubConsumeKafka();
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(consumeKafka);
|
||||
runner.setValidateExpressionUsage(false);
|
||||
runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
|
||||
runner.setProperty(ConsumeKafka.TOPIC, topicName);
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.GROUP_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
|
||||
|
||||
byte[][] values = new byte[][] { "Hello-1".getBytes(StandardCharsets.UTF_8),
|
||||
"Hello-2".getBytes(StandardCharsets.UTF_8), "Hello-3".getBytes(StandardCharsets.UTF_8) };
|
||||
consumeKafka.setValues(values);
|
||||
|
||||
runner.run(1, false);
|
||||
|
||||
values = new byte[][] { "Hello-4".getBytes(StandardCharsets.UTF_8), "Hello-5".getBytes(StandardCharsets.UTF_8),
|
||||
"Hello-6".getBytes(StandardCharsets.UTF_8) };
|
||||
consumeKafka.setValues(values);
|
||||
|
||||
runner.run(1, false);
|
||||
|
||||
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
|
||||
|
||||
assertEquals(6, flowFiles.size());
|
||||
// spot check
|
||||
MockFlowFile flowFile = flowFiles.get(0);
|
||||
String event = new String(flowFile.toByteArray());
|
||||
assertEquals("Hello-1", event);
|
||||
|
||||
flowFile = flowFiles.get(1);
|
||||
event = new String(flowFile.toByteArray());
|
||||
assertEquals("Hello-2", event);
|
||||
|
||||
flowFile = flowFiles.get(5);
|
||||
event = new String(flowFile.toByteArray());
|
||||
assertEquals("Hello-6", event);
|
||||
|
||||
consumeKafka.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateGetAllMessagesWithProvidedDemarcator() throws Exception {
|
||||
String topicName = "validateGetAllMessagesWithProvidedDemarcator";
|
||||
|
||||
StubConsumeKafka consumeKafka = new StubConsumeKafka();
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(consumeKafka);
|
||||
runner.setValidateExpressionUsage(false);
|
||||
runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234");
|
||||
runner.setProperty(ConsumeKafka.TOPIC, topicName);
|
||||
runner.setProperty(ConsumeKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.GROUP_ID, "foo");
|
||||
runner.setProperty(ConsumeKafka.AUTO_OFFSET_RESET, ConsumeKafka.OFFSET_EARLIEST);
|
||||
runner.setProperty(ConsumeKafka.MESSAGE_DEMARCATOR, "blah");
|
||||
|
||||
byte[][] values = new byte[][] { "Hello-1".getBytes(StandardCharsets.UTF_8),
|
||||
"Hi-2".getBytes(StandardCharsets.UTF_8) };
|
||||
consumeKafka.setValues(values);
|
||||
|
||||
runner.run(1, false);
|
||||
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
|
||||
assertEquals(1, flowFiles.size());
|
||||
|
||||
values = new byte[][] { "Здравствуйте-3".getBytes(StandardCharsets.UTF_8),
|
||||
"こんにちは-4".getBytes(StandardCharsets.UTF_8), "Hello-5".getBytes(StandardCharsets.UTF_8) };
|
||||
consumeKafka.setValues(values);
|
||||
|
||||
runner.run(1, false);
|
||||
|
||||
flowFiles = runner.getFlowFilesForRelationship(ConsumeKafka.REL_SUCCESS);
|
||||
|
||||
assertEquals(2, flowFiles.size());
|
||||
MockFlowFile flowFile = flowFiles.get(0);
|
||||
String[] events = new String(flowFile.toByteArray(), StandardCharsets.UTF_8).split("blah");
|
||||
|
||||
assertEquals(2, events.length);
|
||||
|
||||
flowFile = flowFiles.get(1);
|
||||
events = new String(flowFile.toByteArray(), StandardCharsets.UTF_8).split("blah");
|
||||
|
||||
assertEquals(3, events.length);
|
||||
// spot check
|
||||
assertEquals("Здравствуйте-3", events[0]);
|
||||
assertEquals("こんにちは-4", events[1]);
|
||||
assertEquals("Hello-5", events[2]);
|
||||
|
||||
consumeKafka.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,302 @@
|
|||
/*
|
||||
* 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 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.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;
|
||||
|
||||
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);
|
||||
|
||||
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);
|
||||
|
||||
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);
|
||||
|
||||
// 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);
|
||||
|
||||
// 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);
|
||||
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);
|
||||
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("key.serializer", ByteArraySerializer.class.getName());
|
||||
kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName());
|
||||
kafkaProperties.setProperty("bootstrap.servers", "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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,254 @@
|
|||
/*
|
||||
* 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.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.apache.kafka.clients.producer.Producer;
|
||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||
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;
|
||||
|
||||
// The test is valid and should be ran when working on this module. @Ignore is
|
||||
// to speed up the overall build
|
||||
public class PublishKafkaTest {
|
||||
|
||||
@Test
|
||||
public void validatePropertiesValidation() throws Exception {
|
||||
PublishKafka publishKafka = new PublishKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(publishKafka);
|
||||
runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "okeydokey:1234");
|
||||
runner.setProperty(PublishKafka.TOPIC, "foo");
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "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(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.BOOTSTRAP_SERVERS, "localhost:1234");
|
||||
runner.setProperty(PublishKafka.SECURITY_PROTOCOL, PublishKafka.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();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.KEY, "key1");
|
||||
runner.setProperty(PublishKafka.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 validateMultiCharacterDemarcatedMessagesAndCustomPartitioner() {
|
||||
String topicName = "validateMultiCharacterDemarcatedMessagesAndCustomPartitioner";
|
||||
StubPublishKafka putKafka = new StubPublishKafka();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.KEY, "key1");
|
||||
runner.setProperty(PublishKafka.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 validateOnSendFailureAndThenResendSuccess() throws Exception {
|
||||
String topicName = "validateSendFailureAndThenResendSuccess";
|
||||
StubPublishKafka putKafka = new StubPublishKafka();
|
||||
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.KEY, "key1");
|
||||
runner.setProperty(PublishKafka.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 validateOnFutureGetFailureAndThenResendSuccess() throws Exception {
|
||||
String topicName = "validateSendFailureAndThenResendSuccess";
|
||||
StubPublishKafka putKafka = new StubPublishKafka();
|
||||
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.KEY, "key1");
|
||||
runner.setProperty(PublishKafka.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();
|
||||
final TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.KEY, "key1");
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.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();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.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();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.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();
|
||||
TestRunner runner = TestRunners.newTestRunner(putKafka);
|
||||
runner.setProperty(PublishKafka.TOPIC, topicName);
|
||||
runner.setProperty(PublishKafka.CLIENT_ID, "foo");
|
||||
runner.setProperty(PublishKafka.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();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* 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
|
||||
}
|
||||
|
||||
publishingContext.setMaxRequestSize(1024);
|
||||
try {
|
||||
publishingContext.setMaxRequestSize(1024);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
|
||||
try {
|
||||
publishingContext.setMaxRequestSize(-10);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// success
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* 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.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
public class StubConsumeKafka extends ConsumeKafka {
|
||||
|
||||
private byte[][] values;
|
||||
|
||||
public void setValues(byte[][] values) {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected Consumer<byte[], byte[]> buildKafkaResource(ProcessContext context, ProcessSession session) {
|
||||
Consumer<byte[], byte[]> consumer = super.buildKafkaResource(context, session);
|
||||
consumer = mock(Consumer.class);
|
||||
String topicName = context.getProperty(TOPIC).evaluateAttributeExpressions().getValue();
|
||||
|
||||
when(consumer.poll(Mockito.anyLong())).thenAnswer(new Answer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer(InvocationOnMock invocation) throws Throwable {
|
||||
List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
|
||||
for (int i = 0; i < StubConsumeKafka.this.values.length; i++) {
|
||||
byte[] value = StubConsumeKafka.this.values[i];
|
||||
ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(topicName, 0, 0, null, value);
|
||||
records.add(record);
|
||||
}
|
||||
TopicPartition partition = new TopicPartition(topicName, 0);
|
||||
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> m = new LinkedHashMap<>();
|
||||
m.put(partition, records);
|
||||
return new ConsumerRecords<>(m);
|
||||
}
|
||||
});
|
||||
|
||||
return consumer;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
/*
|
||||
* 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.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
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.serialization.ByteArraySerializer;
|
||||
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;
|
||||
|
||||
public Producer<byte[], byte[]> getProducer() {
|
||||
return producer;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected KafkaPublisher buildKafkaResource(ProcessContext context, ProcessSession session)
|
||||
throws ProcessException {
|
||||
Properties kafkaProperties = this.buildKafkaProperties(context);
|
||||
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);
|
||||
producer = mock(Producer.class);
|
||||
this.instrumentProducer(producer, false);
|
||||
Field kf = KafkaPublisher.class.getDeclaredField("kafkaProducer");
|
||||
kf.setAccessible(true);
|
||||
kf.set(publisher, producer);
|
||||
} catch (Exception e) {
|
||||
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>>() {
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
|
||||
ProducerRecord<byte[], byte[]> record = (ProducerRecord<byte[], byte[]>) invocation.getArguments()[0];
|
||||
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 future = mock(Future.class);
|
||||
if ("futurefail".equals(value) && !StubPublishKafka.this.failed) {
|
||||
StubPublishKafka.this.failed = true;
|
||||
when(future.get(Mockito.anyLong(), Mockito.any())).thenThrow(ExecutionException.class);
|
||||
}
|
||||
return future;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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 java.lang.reflect.Modifier;
|
||||
|
||||
import sun.misc.Unsafe;
|
||||
|
||||
class TestUtils {
|
||||
|
||||
public static void setFinalField(Field field, Object instance, Object newValue) throws Exception {
|
||||
field.setAccessible(true);
|
||||
Field modifiersField = Field.class.getDeclaredField("modifiers");
|
||||
modifiersField.setAccessible(true);
|
||||
modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
|
||||
|
||||
field.set(instance, newValue);
|
||||
}
|
||||
|
||||
|
||||
static Unsafe getUnsafe() {
|
||||
try {
|
||||
Field f = Unsafe.class.getDeclaredField("theUnsafe");
|
||||
f.setAccessible(true);
|
||||
return (Unsafe) f.get(null);
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,226 @@
|
|||
/*
|
||||
* 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.File;
|
||||
import java.io.IOException;
|
||||
import java.net.ServerSocket;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.zookeeper.server.ServerCnxnFactory;
|
||||
import org.apache.zookeeper.server.ServerConfig;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
|
||||
import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import kafka.server.KafkaConfig;
|
||||
import kafka.server.KafkaServerStartable;
|
||||
|
||||
/**
|
||||
* Embedded Kafka server, primarily to be used for testing.
|
||||
*/
|
||||
public class EmbeddedKafka {
|
||||
|
||||
private final KafkaServerStartable kafkaServer;
|
||||
|
||||
private final Properties zookeeperConfig;
|
||||
|
||||
private final Properties kafkaConfig;
|
||||
|
||||
private final ZooKeeperServer zkServer;
|
||||
|
||||
private final Logger logger = LoggerFactory.getLogger(EmbeddedKafka.class);
|
||||
|
||||
private final int kafkaPort;
|
||||
|
||||
private final int zookeeperPort;
|
||||
|
||||
private boolean started;
|
||||
|
||||
/**
|
||||
* Will create instance of the embedded Kafka server. Kafka and Zookeeper
|
||||
* configuration properties will be loaded from 'server.properties' and
|
||||
* 'zookeeper.properties' located at the root of the classpath.
|
||||
*/
|
||||
public EmbeddedKafka() {
|
||||
this(loadPropertiesFromClasspath("/server.properties"), loadPropertiesFromClasspath("/zookeeper.properties"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Will create instance of the embedded Kafka server.
|
||||
*
|
||||
* @param kafkaConfig
|
||||
* Kafka configuration properties
|
||||
* @param zookeeperConfig
|
||||
* Zookeeper configuration properties
|
||||
*/
|
||||
public EmbeddedKafka(Properties kafkaConfig, Properties zookeeperConfig) {
|
||||
this.cleanupKafkaWorkDir();
|
||||
this.zookeeperConfig = zookeeperConfig;
|
||||
this.kafkaConfig = kafkaConfig;
|
||||
this.kafkaPort = this.availablePort();
|
||||
this.zookeeperPort = this.availablePort();
|
||||
|
||||
this.kafkaConfig.setProperty("port", String.valueOf(this.kafkaPort));
|
||||
this.kafkaConfig.setProperty("zookeeper.connect", "localhost:" + this.zookeeperPort);
|
||||
this.zookeeperConfig.setProperty("clientPort", String.valueOf(this.zookeeperPort));
|
||||
this.zkServer = new ZooKeeperServer();
|
||||
this.kafkaServer = new KafkaServerStartable(new KafkaConfig(kafkaConfig));
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return port for Kafka server
|
||||
*/
|
||||
public int getKafkaPort() {
|
||||
if (!this.started) {
|
||||
throw new IllegalStateException("Kafka server is not started. Kafka port can't be determined.");
|
||||
}
|
||||
return this.kafkaPort;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return port for Zookeeper server
|
||||
*/
|
||||
public int getZookeeperPort() {
|
||||
if (!this.started) {
|
||||
throw new IllegalStateException("Kafka server is not started. Zookeeper port can't be determined.");
|
||||
}
|
||||
return this.zookeeperPort;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will start embedded Kafka server. Its data directories will be created
|
||||
* at 'kafka-tmp' directory relative to the working directory of the current
|
||||
* runtime. The data directories will be deleted upon JVM exit.
|
||||
*
|
||||
*/
|
||||
public void start() {
|
||||
if (!this.started) {
|
||||
logger.info("Starting Zookeeper server");
|
||||
this.startZookeeper();
|
||||
|
||||
logger.info("Starting Kafka server");
|
||||
this.kafkaServer.startup();
|
||||
|
||||
logger.info("Embeded Kafka is started at localhost:" + this.kafkaServer.serverConfig().port()
|
||||
+ ". Zookeeper connection string: " + this.kafkaConfig.getProperty("zookeeper.connect"));
|
||||
this.started = true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will stop embedded Kafka server, cleaning up all working directories.
|
||||
*/
|
||||
public void stop() {
|
||||
if (this.started) {
|
||||
logger.info("Shutting down Kafka server");
|
||||
this.kafkaServer.shutdown();
|
||||
this.kafkaServer.awaitShutdown();
|
||||
logger.info("Shutting down Zookeeper server");
|
||||
this.shutdownZookeeper();
|
||||
logger.info("Embeded Kafka is shut down.");
|
||||
this.cleanupKafkaWorkDir();
|
||||
this.started = false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
private void cleanupKafkaWorkDir() {
|
||||
File kafkaTmp = new File("target/kafka-tmp");
|
||||
try {
|
||||
FileUtils.deleteDirectory(kafkaTmp);
|
||||
} catch (Exception e) {
|
||||
logger.warn("Failed to delete " + kafkaTmp.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will start Zookeeper server via {@link ServerCnxnFactory}
|
||||
*/
|
||||
private void startZookeeper() {
|
||||
QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
|
||||
try {
|
||||
quorumConfiguration.parseProperties(this.zookeeperConfig);
|
||||
|
||||
ServerConfig configuration = new ServerConfig();
|
||||
configuration.readFrom(quorumConfiguration);
|
||||
|
||||
FileTxnSnapLog txnLog = new FileTxnSnapLog(new File(configuration.getDataLogDir()), new File(configuration.getDataDir()));
|
||||
|
||||
zkServer.setTxnLogFactory(txnLog);
|
||||
zkServer.setTickTime(configuration.getTickTime());
|
||||
zkServer.setMinSessionTimeout(configuration.getMinSessionTimeout());
|
||||
zkServer.setMaxSessionTimeout(configuration.getMaxSessionTimeout());
|
||||
ServerCnxnFactory zookeeperConnectionFactory = ServerCnxnFactory.createFactory();
|
||||
zookeeperConnectionFactory.configure(configuration.getClientPortAddress(),
|
||||
configuration.getMaxClientCnxns());
|
||||
zookeeperConnectionFactory.startup(zkServer);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Failed to start Zookeeper server", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will shut down Zookeeper server.
|
||||
*/
|
||||
private void shutdownZookeeper() {
|
||||
zkServer.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Will load {@link Properties} from properties file discovered at the
|
||||
* provided path relative to the root of the classpath.
|
||||
*/
|
||||
private static Properties loadPropertiesFromClasspath(String path) {
|
||||
try {
|
||||
Properties kafkaProperties = new Properties();
|
||||
kafkaProperties.load(Class.class.getResourceAsStream(path));
|
||||
return kafkaProperties;
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will determine the available port used by Kafka/Zookeeper servers.
|
||||
*/
|
||||
private int availablePort() {
|
||||
ServerSocket s = null;
|
||||
try {
|
||||
s = new ServerSocket(0);
|
||||
s.setReuseAddress(true);
|
||||
return s.getLocalPort();
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Failed to discover available port.", e);
|
||||
} finally {
|
||||
try {
|
||||
s.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
# 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.
|
||||
log4j.rootCategory=INFO, stdout
|
||||
|
||||
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.stdout.layout.ConversionPattern=%d{ABSOLUTE} %5p %t %c{2}:%L - %m%n
|
||||
|
||||
#og4j.category.org.apache.nifi.processors.kafka=DEBUG
|
|
@ -0,0 +1,121 @@
|
|||
# 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.
|
||||
# see kafka.server.KafkaConfig for additional details and defaults
|
||||
|
||||
############################# Server Basics #############################
|
||||
|
||||
# The id of the broker. This must be set to a unique integer for each broker.
|
||||
broker.id=0
|
||||
|
||||
############################# Socket Server Settings #############################
|
||||
|
||||
# The port the socket server listens on
|
||||
#port=9092
|
||||
|
||||
# Hostname the broker will bind to. If not set, the server will bind to all interfaces
|
||||
#host.name=localhost
|
||||
|
||||
# Hostname the broker will advertise to producers and consumers. If not set, it uses the
|
||||
# value for "host.name" if configured. Otherwise, it will use the value returned from
|
||||
# java.net.InetAddress.getCanonicalHostName().
|
||||
#advertised.host.name=<hostname routable by clients>
|
||||
|
||||
# The port to publish to ZooKeeper for clients to use. If this is not set,
|
||||
# it will publish the same port that the broker binds to.
|
||||
#advertised.port=<port accessible by clients>
|
||||
|
||||
# The number of threads handling network requests
|
||||
num.network.threads=3
|
||||
|
||||
# The number of threads doing disk I/O
|
||||
num.io.threads=8
|
||||
|
||||
# The send buffer (SO_SNDBUF) used by the socket server
|
||||
socket.send.buffer.bytes=102400
|
||||
|
||||
# The receive buffer (SO_RCVBUF) used by the socket server
|
||||
socket.receive.buffer.bytes=102400
|
||||
|
||||
# The maximum size of a request that the socket server will accept (protection against OOM)
|
||||
socket.request.max.bytes=104857600
|
||||
|
||||
|
||||
############################# Log Basics #############################
|
||||
|
||||
# A comma seperated list of directories under which to store log files
|
||||
log.dirs=target/kafka-tmp/kafka-logs
|
||||
|
||||
# The default number of log partitions per topic. More partitions allow greater
|
||||
# parallelism for consumption, but this will also result in more files across
|
||||
# the brokers.
|
||||
num.partitions=1
|
||||
|
||||
# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
|
||||
# This value is recommended to be increased for installations with data dirs located in RAID array.
|
||||
num.recovery.threads.per.data.dir=1
|
||||
|
||||
############################# Log Flush Policy #############################
|
||||
|
||||
# Messages are immediately written to the filesystem but by default we only fsync() to sync
|
||||
# the OS cache lazily. The following configurations control the flush of data to disk.
|
||||
# There are a few important trade-offs here:
|
||||
# 1. Durability: Unflushed data may be lost if you are not using replication.
|
||||
# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush.
|
||||
# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks.
|
||||
# The settings below allow one to configure the flush policy to flush data after a period of time or
|
||||
# every N messages (or both). This can be done globally and overridden on a per-topic basis.
|
||||
|
||||
# The number of messages to accept before forcing a flush of data to disk
|
||||
#log.flush.interval.messages=10000
|
||||
|
||||
# The maximum amount of time a message can sit in a log before we force a flush
|
||||
#log.flush.interval.ms=1000
|
||||
|
||||
############################# Log Retention Policy #############################
|
||||
|
||||
# The following configurations control the disposal of log segments. The policy can
|
||||
# be set to delete segments after a period of time, or after a given size has accumulated.
|
||||
# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
|
||||
# from the end of the log.
|
||||
|
||||
# The minimum age of a log file to be eligible for deletion
|
||||
log.retention.hours=168
|
||||
|
||||
# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
|
||||
# segments don't drop below log.retention.bytes.
|
||||
#log.retention.bytes=1073741824
|
||||
|
||||
# The maximum size of a log segment file. When this size is reached a new log segment will be created.
|
||||
log.segment.bytes=1073741824
|
||||
|
||||
# The interval at which log segments are checked to see if they can be deleted according
|
||||
# to the retention policies
|
||||
log.retention.check.interval.ms=300000
|
||||
|
||||
# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
|
||||
# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
|
||||
log.cleaner.enable=false
|
||||
|
||||
############################# Zookeeper #############################
|
||||
|
||||
# Zookeeper connection string (see zookeeper docs for details).
|
||||
# This is a comma separated host:port pairs, each corresponding to a zk
|
||||
# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
|
||||
# You can also append an optional chroot string to the urls to specify the
|
||||
# root directory for all kafka znodes.
|
||||
zookeeper.connect=localhost:2181
|
||||
|
||||
# Timeout in ms for connecting to zookeeper
|
||||
zookeeper.connection.timeout.ms=6000
|
|
@ -0,0 +1,20 @@
|
|||
# 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.
|
||||
# the directory where the snapshot is stored.
|
||||
dataDir=target/kafka-tmp/zookeeper
|
||||
# the port at which the clients will connect
|
||||
#clientPort=2181
|
||||
# disable the per-ip limit on the number of connections since this is a non-production config
|
||||
maxClientCnxns=0
|
|
@ -23,7 +23,9 @@
|
|||
<packaging>pom</packaging>
|
||||
<modules>
|
||||
<module>nifi-kafka-processors</module>
|
||||
<module>nifi-kafka-pubsub-processors</module>
|
||||
<module>nifi-kafka-nar</module>
|
||||
<module>nifi-kafka-pubsub-nar</module>
|
||||
</modules>
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
|
@ -32,6 +34,11 @@
|
|||
<artifactId>nifi-kafka-processors</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-pubsub-processors</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
</project>
|
||||
|
|
6
pom.xml
6
pom.xml
|
@ -964,6 +964,12 @@ language governing permissions and limitations under the License. -->
|
|||
<version>1.0.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-kafka-pubsub-nar</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
<type>nar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-http-context-map-nar</artifactId>
|
||||
|
|
Loading…
Reference in New Issue