mirror of https://github.com/apache/nifi.git
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:
parent
5a84d650c3
commit
c8fc1327ee
|
@ -16,11 +16,31 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kafka.pubsub;
|
||||
|
||||
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 static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
|
||||
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||
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.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.Closeable;
|
||||
import java.io.IOException;
|
||||
|
@ -38,29 +58,10 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import javax.xml.bind.DatatypeConverter;
|
||||
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||
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;
|
||||
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 static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
|
||||
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
|
||||
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);
|
||||
try {
|
||||
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);
|
||||
|
||||
BundleTracker tracker = bundleMap.get(bundleInfo);
|
||||
if (tracker == null) {
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
BundleTracker tracker = bundleMap.get(bundleInfo);
|
||||
if (tracker == null) {
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
final OutputStream rawOut = session.write(flowFile);
|
||||
final OutputStream rawOut = session.write(flowFile);
|
||||
|
||||
final RecordSchema writeSchema;
|
||||
try {
|
||||
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
|
||||
final RecordSchema writeSchema;
|
||||
try {
|
||||
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
|
||||
|
||||
rollback(topicPartition);
|
||||
yield();
|
||||
rollback(topicPartition);
|
||||
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);
|
||||
writer.beginRecordSet();
|
||||
try {
|
||||
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.updateFlowFile(flowFile);
|
||||
bundleMap.put(bundleInfo, tracker);
|
||||
} else {
|
||||
writer = tracker.recordWriter;
|
||||
tracker.incrementRecordCount(1L);
|
||||
session.adjustCounter("Records Received", 1L, false);
|
||||
}
|
||||
|
||||
try {
|
||||
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.incrementRecordCount(1L);
|
||||
session.adjustCounter("Records Received", 1L, false);
|
||||
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
|
||||
handleParseFailure(consumerRecord, session, e);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,11 +16,31 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kafka.pubsub;
|
||||
|
||||
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 static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
|
||||
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||
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.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.Closeable;
|
||||
import java.io.IOException;
|
||||
|
@ -38,29 +58,10 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import javax.xml.bind.DatatypeConverter;
|
||||
|
||||
import org.apache.kafka.clients.consumer.Consumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
|
||||
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;
|
||||
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 static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING;
|
||||
import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING;
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
|
||||
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);
|
||||
try {
|
||||
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);
|
||||
|
||||
BundleTracker tracker = bundleMap.get(bundleInfo);
|
||||
if (tracker == null) {
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
BundleTracker tracker = bundleMap.get(bundleInfo);
|
||||
if (tracker == null) {
|
||||
FlowFile flowFile = session.create();
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
final OutputStream rawOut = session.write(flowFile);
|
||||
final OutputStream rawOut = session.write(flowFile);
|
||||
|
||||
final RecordSchema writeSchema;
|
||||
try {
|
||||
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
|
||||
final RecordSchema writeSchema;
|
||||
try {
|
||||
writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
|
||||
|
||||
rollback(topicPartition);
|
||||
yield();
|
||||
rollback(topicPartition);
|
||||
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);
|
||||
writer.beginRecordSet();
|
||||
try {
|
||||
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.updateFlowFile(flowFile);
|
||||
bundleMap.put(bundleInfo, tracker);
|
||||
} else {
|
||||
writer = tracker.recordWriter;
|
||||
tracker.incrementRecordCount(1L);
|
||||
session.adjustCounter("Records Received", 1L, false);
|
||||
}
|
||||
|
||||
try {
|
||||
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.incrementRecordCount(1L);
|
||||
session.adjustCounter("Records Received", 1L, false);
|
||||
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
|
||||
handleParseFailure(consumerRecord, session, e);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue