exceptionfactory commented on code in PR #7509:
URL: https://github.com/apache/nifi/pull/7509#discussion_r1272582318


##########
nifi-nar-bundles/nifi-jslt-bundle/nifi-jslt-processors/src/main/java/org/apache/nifi/processors/jslt/JSLTTransformJSON.java:
##########
@@ -202,40 +219,73 @@ public void onTrigger(final ProcessContext context, final 
ProcessSession session
             return;
         }
 
+        final TransformStrategy transformStrategy = 
TransformStrategy.valueOf(context.getProperty(TRANSFORM_STRATEGY).getValue());
         final StopWatch stopWatch = new StopWatch(true);
 
-        final JsonNode jsonNode;
-        try (final InputStream in = session.read(original)) {
-            jsonNode = readJson(in);
-        } catch (final Exception e) {
-            getLogger().error("JSLT Transform failed {}", original, e);
-            session.transfer(original, REL_FAILURE);
-            return;
-        }
-
         final PropertyValue transformProperty = 
context.getProperty(JSLT_TRANSFORM);
+        FlowFile transformed;
+        final JsonFactory jsonFactory = new JsonFactory();
 
         try {
             final String transform = readTransform(transformProperty, 
original);
             final Expression jsltExpression = transformCache.get(transform, 
currString -> Parser.compileString(transform));
+            final boolean prettyPrint = 
context.getProperty(PRETTY_PRINT).asBoolean();
 
-            final JsonNode transformedJson = jsltExpression.apply(jsonNode);
-            final ObjectWriter writer = 
context.getProperty(PRETTY_PRINT).asBoolean() ? 
jsonObjectMapper.writerWithDefaultPrettyPrinter() : jsonObjectMapper.writer();
-            final Object outputObject;
-            if (transformedJson == null || transformedJson.isNull()) {
-                getLogger().warn("JSLT Transform resulted in no data {}", 
original);
-                outputObject = null;
-            } else {
-                outputObject = transformedJson;
-            }
-            FlowFile transformed = session.write(original, out -> {
-                if (outputObject != null) {
-                    writer.writeValue(out, outputObject);
+            transformed = session.write(original, (inputStream, outputStream) 
-> {
+                boolean topLevelArray = false;
+                JsonParser jsonParser;
+                JsonNode firstJsonNode;
+                if (APPLY_TRANSFORM_TO_EACH_OBJECT.equals(transformStrategy)) {
+                    jsonParser = jsonFactory.createParser(inputStream);
+                    jsonParser.setCodec(jsonObjectMapper);
+
+                    JsonToken token = jsonParser.nextToken();
+                    if (token == JsonToken.START_ARRAY) {
+                        token = jsonParser.nextToken(); // advance to 
START_OBJECT token
+                        topLevelArray = true;
+                    }
+                    if (token == JsonToken.START_OBJECT) { // could be 
END_ARRAY also
+                        firstJsonNode = jsonParser.readValueAsTree();
+                    } else {
+                        firstJsonNode = null;
+                    }
+                } else {
+                    firstJsonNode = readJson(inputStream);
+                    jsonParser = null; // This will not be used when applying 
the transform to the entire FlowFile
+                }
+
+                final ObjectWriter writer = prettyPrint ? 
jsonObjectMapper.writerWithDefaultPrettyPrinter() : jsonObjectMapper.writer();
+                final JsonGenerator jsonGenerator = 
writer.createGenerator(outputStream);
+
+                Object outputObject;
+                JsonNode nextNode;
+
+                if (topLevelArray) {
+                    jsonGenerator.writeStartArray();
                 }
+                nextNode = firstJsonNode;
+                do {
+                    final JsonNode transformedJson = 
jsltExpression.apply(nextNode);
+                    if (transformedJson == null || transformedJson.isNull()) {
+                        getLogger().warn("JSLT Transform resulted in no data 
{}", original);

Review Comment:
   This could be repetitive with a large number of records, perhaps set a flag 
an log once after the loop?



##########
nifi-nar-bundles/nifi-jslt-bundle/nifi-jslt-processors/src/main/java/org/apache/nifi/processors/jslt/JSLTTransformJSON.java:
##########
@@ -277,4 +327,68 @@ private String readTransform(final PropertyValue 
propertyValue) {
             throw new UncheckedIOException("Read JSLT Transform failed", e);
         }
     }
-}
+
+
+    protected JsonNode getNextJsonNode(final TransformStrategy 
transformStrategy, final JsonParser jsonParser) throws IOException {
+
+        if (APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE.equals(transformStrategy)) {
+            return null;
+        }
+        return getJsonNode(jsonParser);
+    }
+
+    private JsonNode getJsonNode(JsonParser jsonParser) throws IOException {
+        while (true) {
+            final JsonToken token = jsonParser.nextToken();
+            if (token == null) {
+                return null;
+            }
+
+            switch (token) {
+                case START_ARRAY:
+                case END_ARRAY:
+                case END_OBJECT:
+                    break;
+                case START_OBJECT:
+                    return jsonParser.readValueAsTree();
+                default:
+                    throw new IOException("Expected to get a JSON Object but 
got a token of type " + token.name());
+            }
+        }
+    }
+
+    /**
+     * Enumeration of supported Output Strategies
+     */
+    enum TransformStrategy implements DescribedValue {
+        
APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE("APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE", 
"Entire FlowFile", "Entire FlowFile"),
+        APPLY_TRANSFORM_TO_EACH_OBJECT("APPLY_TRANSFORM_TO_EACH_OBJECT", "Each 
JSON Object", "Each JSON Object");
+
+        private final String value;
+
+        private final String displayName;
+
+        private final String description;
+
+        TransformStrategy(final String value, final String displayName, final 
String description) {
+            this.value = value;
+            this.displayName = displayName;
+            this.description = description;
+        }
+
+        @Override
+        public String getValue() {
+            return value;

Review Comment:
   ```suggestion
               return name();
   ```



##########
nifi-nar-bundles/nifi-jslt-bundle/nifi-jslt-processors/src/main/java/org/apache/nifi/processors/jslt/JSLTTransformJSON.java:
##########
@@ -277,4 +327,68 @@ private String readTransform(final PropertyValue 
propertyValue) {
             throw new UncheckedIOException("Read JSLT Transform failed", e);
         }
     }
-}
+
+
+    protected JsonNode getNextJsonNode(final TransformStrategy 
transformStrategy, final JsonParser jsonParser) throws IOException {
+
+        if (APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE.equals(transformStrategy)) {
+            return null;
+        }
+        return getJsonNode(jsonParser);
+    }
+
+    private JsonNode getJsonNode(JsonParser jsonParser) throws IOException {
+        while (true) {
+            final JsonToken token = jsonParser.nextToken();
+            if (token == null) {
+                return null;
+            }
+
+            switch (token) {
+                case START_ARRAY:
+                case END_ARRAY:
+                case END_OBJECT:
+                    break;
+                case START_OBJECT:
+                    return jsonParser.readValueAsTree();
+                default:
+                    throw new IOException("Expected to get a JSON Object but 
got a token of type " + token.name());
+            }
+        }
+    }
+
+    /**
+     * Enumeration of supported Output Strategies
+     */
+    enum TransformStrategy implements DescribedValue {
+        
APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE("APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE", 
"Entire FlowFile", "Entire FlowFile"),
+        APPLY_TRANSFORM_TO_EACH_OBJECT("APPLY_TRANSFORM_TO_EACH_OBJECT", "Each 
JSON Object", "Each JSON Object");
+
+        private final String value;

Review Comment:
   The value string does not need to be repeated and the `name()` method can be 
used inside `getValue()`:
   ```suggestion
           APPLY_TRANSFORM_TO_ENTIRE_FLOWFILE("Entire FlowFile", "Entire 
FlowFile"),
           APPLY_TRANSFORM_TO_EACH_OBJECT("Each JSON Object", "Each JSON 
Object");
   ```



-- 
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: issues-unsubscr...@nifi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to