NIFI-13283 - Fix exception thrown in PutKinesisFirehose processor by ensuring value exists before referencing

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

This closes #8867.
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
index 46ba510..3f81007 100644
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
@@ -130,9 +130,9 @@
             for (final FlowFile flowFile : flowFiles) {
                 final String firehoseStreamName = context.getProperty(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();
 
+                recordHash.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
                 session.read(flowFile, in -> recordHash.get(firehoseStreamName).add(Record.builder().data(SdkBytes.fromInputStream(in)).build()));
 
-                recordHash.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
                 final List<FlowFile> flowFilesForStream = hashFlowFiles.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
                 flowFilesForStream.add(flowFile);
             }