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;
|
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,6 +531,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
Record record;
|
Record record;
|
||||||
while ((record = reader.nextRecord()) != null) {
|
while ((record = reader.nextRecord()) != null) {
|
||||||
// Determine the bundle for this record.
|
// Determine the bundle for this record.
|
||||||
|
@ -576,6 +578,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
|
||||||
tracker.incrementRecordCount(1L);
|
tracker.incrementRecordCount(1L);
|
||||||
session.adjustCounter("Records Received", 1L, false);
|
session.adjustCounter("Records Received", 1L, false);
|
||||||
}
|
}
|
||||||
|
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
|
||||||
|
handleParseFailure(consumerRecord, session, e);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
|
|
|
@ -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,6 +531,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
Record record;
|
Record record;
|
||||||
while ((record = reader.nextRecord()) != null) {
|
while ((record = reader.nextRecord()) != null) {
|
||||||
// Determine the bundle for this record.
|
// Determine the bundle for this record.
|
||||||
|
@ -576,6 +578,10 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
|
||||||
tracker.incrementRecordCount(1L);
|
tracker.incrementRecordCount(1L);
|
||||||
session.adjustCounter("Records Received", 1L, false);
|
session.adjustCounter("Records Received", 1L, false);
|
||||||
}
|
}
|
||||||
|
} catch (final IOException | MalformedRecordException | SchemaValidationException e) {
|
||||||
|
handleParseFailure(consumerRecord, session, e);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
|
|
Loading…
Reference in New Issue