NIFI-4008: This closes #2189. Update ConsumeKafkaRecord 0.11 so that it can consume multiple records from a single Kafka message

NIFI-4008: Ensure that we always check if a Kafka message's value is null before dereferencing it

Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
Mark Payne 2017-10-02 16:02:54 -04:00 committed by joewitt
parent 1d5df4a5ec
commit 582df7f4e8
2 changed files with 60 additions and 57 deletions

View File

@ -446,7 +446,9 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
attributes.put(KafkaProcessorUtils.KAFKA_TOPIC, topicPartition.topic());
FlowFile failureFlowFile = session.create();
if (consumerRecord.value() != null) {
failureFlowFile = session.write(failureFlowFile, out -> out.write(consumerRecord.value()));
}
failureFlowFile = session.putAllAttributes(failureFlowFile, attributes);
final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, topicPartition.topic());
@ -461,7 +463,8 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
try {
for (final ConsumerRecord<byte[], byte[]> consumerRecord : messages) {
try (final InputStream in = new ByteArrayInputStream(consumerRecord.value())) {
final byte[] recordBytes = consumerRecord.value() == null ? new byte[0] : consumerRecord.value();
try (final InputStream in = new ByteArrayInputStream(recordBytes)) {
final RecordReader reader;
final Record firstRecord;

View File

@ -512,20 +512,19 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
for (final ConsumerRecord<byte[], byte[]> consumerRecord : records) {
final Map<String, String> attributes = getAttributes(consumerRecord);
final Record record;
try (final InputStream in = new ByteArrayInputStream(consumerRecord.value())) {
final RecordReader reader = readerFactory.createRecordReader(attributes, in, logger);
record = reader.nextRecord();
final byte[] recordBytes = consumerRecord.value() == null ? new byte[0] : consumerRecord.value();
try (final InputStream in = new ByteArrayInputStream(recordBytes)) {
final RecordReader reader;
try {
reader = readerFactory.createRecordReader(attributes, in, logger);
} catch (final Exception e) {
handleParseFailure(consumerRecord, session, e);
continue;
}
if (record == null) {
handleParseFailure(consumerRecord, session, null);
continue;
}
Record record;
while ((record = reader.nextRecord()) != null) {
// Determine the bundle for this record.
final RecordSchema recordSchema = record.getSchema();
final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes);
@ -572,9 +571,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
}
tracker.incrementRecordCount(1L);
}
session.adjustCounter("Records Received", records.size(), false);
}
}
}
} catch (final Exception e) {
logger.error("Failed to properly receive messages from Kafka. Will roll back session and any un-committed offsets from Kafka.", e);