sfc-gh-mpayne commented on code in PR #10077:
URL: https://github.com/apache/nifi/pull/10077#discussion_r2399496590


##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.aws.kinesis;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+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.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Collections.emptyMap;
+import static 
org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.MIME_TYPE;
+import static 
org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT;
+import static 
org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE;
+
+final class ReaderRecordProcessor {
+
+    private final RecordReaderFactory recordReaderFactory;
+    private final RecordSetWriterFactory recordWriterFactory;
+    private final ComponentLog logger;
+
+    ReaderRecordProcessor(
+            final RecordReaderFactory recordReaderFactory,
+            final RecordSetWriterFactory recordWriterFactory,
+            final ComponentLog logger) {
+        this.recordReaderFactory = recordReaderFactory;
+        this.recordWriterFactory = recordWriterFactory;
+        this.logger = logger;
+    }
+
+    ProcessingResult processRecords(
+            final ProcessSession session,
+            final String streamName,
+            final String shardId,
+            final List<KinesisClientRecord> records) {
+        final List<FlowFile> successFlowFiles = new ArrayList<>();
+        final List<FlowFile> failureFlowFiles = new ArrayList<>();
+
+        ActiveFlowFile activeFlowFile = null;
+
+        for (final KinesisClientRecord kinesisRecord : records) {
+            final int dataSize = kinesisRecord.data().remaining();
+            final byte[] data = new byte[dataSize];
+            kinesisRecord.data().get(data);
+
+            try (final InputStream in = new ByteArrayInputStream(data);
+                 final RecordReader reader = 
recordReaderFactory.createRecordReader(emptyMap(), in, data.length, logger)) {
+
+                Record record;
+                while ((record = reader.nextRecord()) != null) {
+                    final RecordSchema writeSchema = 
recordWriterFactory.getSchema(emptyMap(), record.getSchema());
+
+                    if (activeFlowFile == null) {
+                        activeFlowFile = ActiveFlowFile.startNewFile(logger, 
session, recordWriterFactory, writeSchema, streamName, shardId);
+                    } else if (!writeSchema.equals(activeFlowFile.schema())) {
+                        // If the write schema has changed, we need to 
complete the current FlowFile and start a new one.
+                        final FlowFile completedFlowFile = 
activeFlowFile.complete();
+                        successFlowFiles.add(completedFlowFile);
+
+                        activeFlowFile = ActiveFlowFile.startNewFile(logger, 
session, recordWriterFactory, writeSchema, streamName, shardId);
+                    }
+
+                    activeFlowFile.writeRecord(record, kinesisRecord);
+                }
+            } catch (final IOException | MalformedRecordException | 
SchemaNotFoundException e) {
+                logger.error("Reader or Writer failed to process Kinesis 
Record with Stream Name [{}] Shard Id [{}] Sequence Number [{}] SubSequence 
Number [{}]",
+                        streamName, shardId, kinesisRecord.sequenceNumber(), 
kinesisRecord.subSequenceNumber(), e);
+                final FlowFile failureFlowFile = 
createParseFailureFlowFile(session, streamName, shardId, kinesisRecord, e);
+                failureFlowFiles.add(failureFlowFile);
+            }
+        }
+
+        if (activeFlowFile != null) {
+            final FlowFile completedFlowFile = activeFlowFile.complete();
+            successFlowFiles.add(completedFlowFile);
+        }
+
+        return new ProcessingResult(successFlowFiles, failureFlowFiles);
+    }
+
+    private static FlowFile createParseFailureFlowFile(
+            final ProcessSession session,
+            final String streamName,
+            final String shardId,
+            final KinesisClientRecord record,
+            final Exception e) {
+        FlowFile flowFile = session.create();
+
+        record.data().rewind();
+        flowFile = session.write(flowFile, out -> 
Channels.newChannel(out).write(record.data()));
+
+        flowFile = session.putAllAttributes(flowFile, 
ConsumeKinesisAttributes.fromKinesisRecords(streamName, shardId, record, 
record));
+
+        final Throwable cause = e.getCause() != null ? e.getCause() : e;
+        final String errorMessage = cause.getLocalizedMessage() != null ? 
cause.getLocalizedMessage() : "NiFi Reader or Writer failed to process Kinesis 
Record";
+        flowFile = session.putAttribute(flowFile, RECORD_ERROR_MESSAGE, 
errorMessage);

Review Comment:
   I agree that just calling `toString()` on the Exception is less than ideal 
and leaks internal details. But the message itself without the Exception name 
is generally not helpful - especially when looking at the `cause`. For example, 
a MalformedRecordException may have been thrown due to a 
`NumberFormatException` for example. In that case, your message might be 
something like `SEVEN` - entirely meaningless without the fact that it's a 
`NumberFormatException`. Now, given that this is called only when one of 3 
Exceptions are caught, we could also just provide a meaningful message along 
with it, so instead of including Exception.toString() we could include a value 
such as `"Malformed Record: " + errorMesage`, `"Schema Not Found: " + 
errorMessage`, etc. But using the Exception's message alone is not a good idea 
unless we know for sure that the message itself is meaningful on its own (such 
as when we know there are only 3 cases the Exception can be thrown and in each 
case we include a m
 eaningful message). But that's not the case here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to