NIFI-5592: If an Exception is thrown by RecordReader.read() from ConsumeKafkaRecord, route Record to parse.failure relationship

Signed-off-by: Pierre Villard <pierre.villard.fr@gmail.com>

This closes #3001.
This commit is contained in:
Mark Payne 2018-09-13 13:56:13 -04:00 committed by Pierre Villard
parent 5a84d650c3
commit c8fc1327ee
2 changed files with 138 additions and 126 deletions

View File

@ -16,11 +16,31 @@
*/ */
package org.apache.nifi.processors.kafka.pubsub; package org.apache.nifi.processors.kafka.pubsub;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11.REL_PARSE_FAILURE; import org.apache.kafka.clients.consumer.Consumer;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11.REL_SUCCESS; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; import org.apache.kafka.clients.consumer.ConsumerRecord;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Header;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.SchemaValidationException;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
import javax.xml.bind.DatatypeConverter;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -38,29 +58,10 @@ import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import javax.xml.bind.DatatypeConverter; import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11.REL_PARSE_FAILURE;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_0_11.REL_SUCCESS;
import org.apache.kafka.clients.consumer.Consumer; import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
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.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Header;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
/** /**
* This class represents a lease to access a Kafka Consumer object. The lease is * This class represents a lease to access a Kafka Consumer object. The lease is
@ -530,51 +531,56 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
continue; continue;
} }
Record record; try {
while ((record = reader.nextRecord()) != null) { Record record;
// Determine the bundle for this record. while ((record = reader.nextRecord()) != null) {
final RecordSchema recordSchema = record.getSchema(); // Determine the bundle for this record.
final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes); final RecordSchema recordSchema = record.getSchema();
final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes);
BundleTracker tracker = bundleMap.get(bundleInfo); BundleTracker tracker = bundleMap.get(bundleInfo);
if (tracker == null) { if (tracker == null) {
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.putAllAttributes(flowFile, attributes); flowFile = session.putAllAttributes(flowFile, attributes);
final OutputStream rawOut = session.write(flowFile); final OutputStream rawOut = session.write(flowFile);
final RecordSchema writeSchema; final RecordSchema writeSchema;
try { try {
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema); writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
} catch (final Exception e) { } catch (final Exception e) {
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e); logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
rollback(topicPartition); rollback(topicPartition);
yield(); yield();
throw new ProcessException(e); throw new ProcessException(e);
}
writer = writerFactory.createWriter(logger, writeSchema, rawOut);
writer.beginRecordSet();
tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer);
tracker.updateFlowFile(flowFile);
bundleMap.put(bundleInfo, tracker);
} else {
writer = tracker.recordWriter;
} }
writer = writerFactory.createWriter(logger, writeSchema, rawOut); try {
writer.beginRecordSet(); writer.write(record);
} catch (final RuntimeException re) {
handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
+ "Will route message as its own FlowFile to the 'parse.failure' relationship");
continue;
}
tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer); tracker.incrementRecordCount(1L);
tracker.updateFlowFile(flowFile); session.adjustCounter("Records Received", 1L, false);
bundleMap.put(bundleInfo, tracker);
} else {
writer = tracker.recordWriter;
} }
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
try { handleParseFailure(consumerRecord, session, e);
writer.write(record); continue;
} catch (final RuntimeException re) {
handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
+ "Will route message as its own FlowFile to the 'parse.failure' relationship");
continue;
}
tracker.incrementRecordCount(1L);
session.adjustCounter("Records Received", 1L, false);
} }
} }
} }

View File

@ -16,11 +16,31 @@
*/ */
package org.apache.nifi.processors.kafka.pubsub; package org.apache.nifi.processors.kafka.pubsub;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_PARSE_FAILURE; import org.apache.kafka.clients.consumer.Consumer;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_SUCCESS; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; import org.apache.kafka.clients.consumer.ConsumerRecord;
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Header;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.SchemaValidationException;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
import javax.xml.bind.DatatypeConverter;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -38,29 +58,10 @@ import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import javax.xml.bind.DatatypeConverter; import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_PARSE_FAILURE;
import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_1_0.REL_SUCCESS;
import org.apache.kafka.clients.consumer.Consumer; import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
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.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Header;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
/** /**
* This class represents a lease to access a Kafka Consumer object. The lease is * This class represents a lease to access a Kafka Consumer object. The lease is
@ -530,51 +531,56 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
continue; continue;
} }
Record record; try {
while ((record = reader.nextRecord()) != null) { Record record;
// Determine the bundle for this record. while ((record = reader.nextRecord()) != null) {
final RecordSchema recordSchema = record.getSchema(); // Determine the bundle for this record.
final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes); final RecordSchema recordSchema = record.getSchema();
final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes);
BundleTracker tracker = bundleMap.get(bundleInfo); BundleTracker tracker = bundleMap.get(bundleInfo);
if (tracker == null) { if (tracker == null) {
FlowFile flowFile = session.create(); FlowFile flowFile = session.create();
flowFile = session.putAllAttributes(flowFile, attributes); flowFile = session.putAllAttributes(flowFile, attributes);
final OutputStream rawOut = session.write(flowFile); final OutputStream rawOut = session.write(flowFile);
final RecordSchema writeSchema; final RecordSchema writeSchema;
try { try {
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema); writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
} catch (final Exception e) { } catch (final Exception e) {
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e); logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
rollback(topicPartition); rollback(topicPartition);
yield(); yield();
throw new ProcessException(e); throw new ProcessException(e);
}
writer = writerFactory.createWriter(logger, writeSchema, rawOut);
writer.beginRecordSet();
tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer);
tracker.updateFlowFile(flowFile);
bundleMap.put(bundleInfo, tracker);
} else {
writer = tracker.recordWriter;
} }
writer = writerFactory.createWriter(logger, writeSchema, rawOut); try {
writer.beginRecordSet(); writer.write(record);
} catch (final RuntimeException re) {
handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
+ "Will route message as its own FlowFile to the 'parse.failure' relationship");
continue;
}
tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer); tracker.incrementRecordCount(1L);
tracker.updateFlowFile(flowFile); session.adjustCounter("Records Received", 1L, false);
bundleMap.put(bundleInfo, tracker);
} else {
writer = tracker.recordWriter;
} }
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
try { handleParseFailure(consumerRecord, session, e);
writer.write(record); continue;
} catch (final RuntimeException re) {
handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
+ "Will route message as its own FlowFile to the 'parse.failure' relationship");
continue;
}
tracker.incrementRecordCount(1L);
session.adjustCounter("Records Received", 1L, false);
} }
} }
} }