You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2022/11/30 19:53:44 UTC

[nifi] 06/22: NIFI-10873 - GenerateFlowFile: flowfiles in a batch are not unique

This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.19
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e0b4feac575a9c241b19bacdfd3d166015d43eda
Author: Arpad Boda <ab...@apache.org>
AuthorDate: Thu Nov 24 16:26:31 2022 +0100

    NIFI-10873 - GenerateFlowFile: flowfiles in a batch are not unique
    
    This closes #6717.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../nifi/processors/standard/GenerateFlowFile.java | 22 +++++++++++++---------
 1 file changed, 13 insertions(+), 9 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
index 0d323e68cf..5c9b7cffd3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java
@@ -219,13 +219,16 @@ public class GenerateFlowFile extends AbstractProcessor {
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         final byte[] data;
-        if (context.getProperty(UNIQUE_FLOWFILES).asBoolean()) {
-            data = generateData(context);
-        } else if(context.getProperty(CUSTOM_TEXT).isSet()) {
-            final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
-            data = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue().getBytes(charset);
+        final boolean uniqueData = context.getProperty(UNIQUE_FLOWFILES).asBoolean();
+        if (uniqueData) {
+            data = new byte[0];
         } else {
-            data = this.data.get();
+            if (context.getProperty(CUSTOM_TEXT).isSet()) {
+                final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
+                data = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue().getBytes(charset);
+            } else {
+                data = this.data.get();
+            }
         }
 
         Map<PropertyDescriptor, String> processorProperties = context.getProperties();
@@ -243,12 +246,13 @@ public class GenerateFlowFile extends AbstractProcessor {
         }
 
         for (int i = 0; i < context.getProperty(BATCH_SIZE).asInteger(); i++) {
-            FlowFile flowFile = session.create();
-            if (data.length > 0) {
+        FlowFile flowFile = session.create();
+            final byte[] writtenData = uniqueData ? generateData(context) : data;
+            if (writtenData.length > 0) {
                 flowFile = session.write(flowFile, new OutputStreamCallback() {
                     @Override
                     public void process(final OutputStream out) throws IOException {
-                        out.write(data);
+                        out.write(writtenData);
                     }
                 });
             }