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 2015/02/18 05:30:03 UTC

[01/29] incubator-nifi git commit: Adding JSONPath as a managed dependency and including it in the standard processor pom

Repository: incubator-nifi
Updated Branches:
  refs/heads/NIFI-360 [created] 286277123


Adding JSONPath as a managed dependency and including it in the standard processor pom


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/446fc2ef
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/446fc2ef
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/446fc2ef

Branch: refs/heads/NIFI-360
Commit: 446fc2ef8099faff786f11c7d640232191dac2b0
Parents: b8ade5b
Author: Aldrin Piri <al...@gmail.com>
Authored: Fri Feb 13 23:42:40 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Fri Feb 13 23:42:40 2015 -0500

----------------------------------------------------------------------
 .../nifi-standard-bundle/nifi-standard-processors/pom.xml       | 4 ++++
 nifi/pom.xml                                                    | 5 +++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/446fc2ef/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index a482e19..5a15cf4 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -151,6 +151,10 @@
             <artifactId>activemq-client</artifactId>
         </dependency>
         <dependency>
+            <groupId>com.jayway.jsonpath</groupId>
+            <artifactId>json-path</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-ssl-context-service</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/446fc2ef/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 8539ae9..20f3a42 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -783,6 +783,11 @@
                 <artifactId>nifi-write-ahead-log</artifactId>
                 <version>0.0.2-incubating-SNAPSHOT</version>
             </dependency>
+            <dependency>
+                <groupId>com.jayway.jsonpath</groupId>
+                <artifactId>json-path</artifactId>
+                <version>1.2.0</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
     <dependencies>


[13/29] incubator-nifi git commit: Adding tests for indefinite results with and without the usage of operators.

Posted by jo...@apache.org.
Adding tests for indefinite results with and without the usage of operators.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/7e581307
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/7e581307
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/7e581307

Branch: refs/heads/NIFI-360
Commit: 7e581307b7d65da79464914b09fe8a90dd5046e9
Parents: 6897090
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:33:48 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:33:48 2015 -0500

----------------------------------------------------------------------
 .../standard/TestEvaluateJsonPath.java          | 35 ++++++++++++++++++++
 1 file changed, 35 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7e581307/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index fc125c2..bece3da 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -177,4 +177,39 @@ public class TestEvaluateJsonPath {
         testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
         testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals("54df94072d5dbf7dc6340cc5");
     }
+
+
+    @Test
+    public void testExtractPath_destinationAttribute_indefiniteResult() throws Exception {
+        String jsonPathAttrKey = "friends.indefinite.id.list";
+
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty(jsonPathAttrKey, "$[0].friends.[*].id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals("[0,1,2]");
+    }
+
+    @Test
+    public void testExtractPath_destinationAttribute_indefiniteResult_operators() throws Exception {
+        String jsonPathAttrKey = "friends.indefinite.id.list";
+
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty(jsonPathAttrKey, "$[0].friends[?(@.id < 3)].id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals("[0,1,2]");
+    }
 }


[03/29] incubator-nifi git commit: Stubbing out validator, setting up relationships and properties.

Posted by jo...@apache.org.
Stubbing out validator, setting up relationships and properties.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/ec669e5b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/ec669e5b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/ec669e5b

Branch: refs/heads/NIFI-360
Commit: ec669e5b42e58335ab511dfa8dfd52bd0848f088
Parents: 5f03fb1
Author: Aldrin Piri <al...@gmail.com>
Authored: Sat Feb 14 13:26:08 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Sat Feb 14 13:26:08 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJSONPath.java   | 95 +++++++++++++++++---
 1 file changed, 85 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ec669e5b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
index 9ad3e49..349e623 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
@@ -22,38 +22,113 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
 
+import java.util.*;
+
 @EventDriven
 @SideEffectFree
 @SupportsBatching
-@Tags({"JSON", "evaluate", "JSONPath"})
+@Tags({"JSON", "evaluate", "JsonPath"})
 @CapabilityDescription("")
-public class EvaluateJSONPath extends AbstractProcessor {
+public class EvaluateJsonPath extends AbstractProcessor {
 
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
     public static final String DESTINATION_CONTENT = "flowfile-content";
 
     public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
             .name("Destination")
-            .description("Indicates whether the results of the JSONPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JSONPath may be specified, and the property name is ignored.")
+            .description("Indicates whether the results of the JsonPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JsonPath may be specified, and the property name is ignored.")
             .required(true)
             .allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTE)
             .defaultValue(DESTINATION_CONTENT)
             .build();
 
-    public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JSONPath is successfully evaluated and the FlowFile is modified as a result").build();
-    public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JSONPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JSONPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
+    public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
+    public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JsonPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JsonPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
+
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> properties;
+
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_MATCH);
+        relationships.add(REL_NO_MATCH);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DESTINATION);
+        this.properties = Collections.unmodifiableList(properties);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
+
+        final String destination = context.getProperty(DESTINATION).getValue();
+        if (DESTINATION_CONTENT.equals(destination)) {
+            int jsonPathCount = 0;
+
+            for (final PropertyDescriptor desc : context.getProperties().keySet()) {
+                if (desc.isDynamic()) {
+                    jsonPathCount++;
+                }
+            }
+
+            if (jsonPathCount != 1) {
+                results.add(new ValidationResult.Builder().subject("JsonPaths").valid(false).explanation("Exactly one JsonPath must be set if using destination of " + DESTINATION_CONTENT).build());
+            }
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
 
 
     @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(false)
+                .addValidator(new JsonPathValidator())
+                .required(false)
+                .dynamic(true)
+                .build();
+    }
+
+    @Override
     public void onTrigger(ProcessContext processContext, ProcessSession processSession) throws ProcessException {
 
+        final FlowFile flowFile = processSession.get();
+        if (flowFile == null) {
+            return;
+        }
+
     }
 
+    private static class JsonPathValidator implements Validator {
+
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            return null;
+        }
+    }
 }


[04/29] incubator-nifi git commit: Adding EvaluateJsonPath to the Processor services file

Posted by jo...@apache.org.
Adding EvaluateJsonPath to the Processor services file


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e75213ee
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e75213ee
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e75213ee

Branch: refs/heads/NIFI-360
Commit: e75213eead936baa7046d9c9ba5b86a4508f532a
Parents: ec669e5
Author: Aldrin Piri <al...@gmail.com>
Authored: Sat Feb 14 13:35:53 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Sat Feb 14 13:35:53 2015 -0500

----------------------------------------------------------------------
 .../resources/META-INF/services/org.apache.nifi.processor.Processor | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e75213ee/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 66c9697..8a1fd74 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -19,6 +19,7 @@ org.apache.nifi.processors.standard.ConvertCharacterSet
 org.apache.nifi.processors.standard.DetectDuplicate
 org.apache.nifi.processors.standard.DistributeLoad
 org.apache.nifi.processors.standard.EncryptContent
+org.apache.nifi.processors.standard.EvaluateJsonPath
 org.apache.nifi.processors.standard.EvaluateRegularExpression
 org.apache.nifi.processors.standard.EvaluateXPath
 org.apache.nifi.processors.standard.EvaluateXQuery


[18/29] incubator-nifi git commit: Refactoring common JSON/JsonPath functionality into JsonUtils

Posted by jo...@apache.org.
Refactoring common JSON/JsonPath functionality into JsonUtils


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/59ad1948
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/59ad1948
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/59ad1948

Branch: refs/heads/NIFI-360
Commit: 59ad1948519fa1f8acccfb6eafbda04684edbc96
Parents: 5a81f19
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 10:26:18 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 10:26:18 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   |  52 ++--------
 .../processors/standard/util/JsonPathUtils.java |  44 --------
 .../processors/standard/util/JsonUtils.java     | 104 +++++++++++++++++++
 3 files changed, 111 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59ad1948/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 05d31da..0bdd20c 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -16,12 +16,9 @@
  */
 package org.apache.nifi.processors.standard;
 
-import com.jayway.jsonpath.Configuration;
 import com.jayway.jsonpath.DocumentContext;
 import com.jayway.jsonpath.JsonPath;
 import com.jayway.jsonpath.PathNotFoundException;
-import com.jayway.jsonpath.spi.json.JsonProvider;
-import net.minidev.json.JSONValue;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -34,17 +31,12 @@ import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.processors.standard.util.JsonPathUtils;
-import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.processors.standard.util.JsonUtils;
 import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.util.BooleanHolder;
 import org.apache.nifi.util.ObjectHolder;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.*;
@@ -95,8 +87,6 @@ public class EvaluateJsonPath extends AbstractProcessor {
     private Set<Relationship> relationships;
     private List<PropertyDescriptor> properties;
 
-    private static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider();
-
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
@@ -150,7 +140,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .expressionLanguageSupported(false)
-                .addValidator(JsonPathUtils.JSON_PATH_VALIDATOR)
+                .addValidator(JsonUtils.JSON_PATH_VALIDATOR)
                 .required(false)
                 .dynamic(true)
                 .build();
@@ -182,42 +172,15 @@ public class EvaluateJsonPath extends AbstractProcessor {
 
         flowFileLoop:
         for (FlowFile flowFile : flowFiles) {
-            // Validate the JSON document before attempting processing
-            final BooleanHolder validJsonHolder = new BooleanHolder(false);
-            processSession.read(flowFile, new InputStreamCallback() {
-                @Override
-                public void process(InputStream in) throws IOException {
-                    try (InputStreamReader inputStreamReader = new InputStreamReader(in)) {
-                        /*
-                         * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
-                         * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
-                         */
-                        boolean validJson = JSONValue.isValidJsonStrict(inputStreamReader);
-                        validJsonHolder.set(validJson);
-                    }
-                }
-            });
 
-            if (!validJsonHolder.get()) {
-                logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile.getId()});
+            final DocumentContext documentContext = JsonUtils.validateAndEstablishJsonContext(processSession, flowFile);
+
+            if (documentContext == null) {
+                logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile});
                 processSession.transfer(flowFile, REL_FAILURE);
                 continue flowFileLoop;
             }
 
-            // Parse the document once into an associated context to support multiple path evaluations if specified
-            final ObjectHolder<DocumentContext> contextHolder = new ObjectHolder<>(null);
-            processSession.read(flowFile, new InputStreamCallback() {
-                @Override
-                public void process(InputStream in) throws IOException {
-                    try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) {
-                        DocumentContext ctx = JsonPath.parse(in);
-                        contextHolder.set(ctx);
-                    }
-                }
-            });
-
-            final DocumentContext documentContext = contextHolder.get();
-
             final Map<String, String> jsonPathResults = new HashMap<>();
 
             jsonPathEvalLoop:
@@ -226,7 +189,6 @@ public class EvaluateJsonPath extends AbstractProcessor {
                 String jsonPathAttrKey = attributeJsonPathEntry.getKey();
                 JsonPath jsonPathExp = attributeJsonPathEntry.getValue();
 
-
                 final ObjectHolder<Object> resultHolder = new ObjectHolder<>(null);
                 try {
                     Object result = documentContext.read(jsonPathExp);
@@ -274,7 +236,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
         if (isScalar(jsonPathResult)) {
             return jsonPathResult.toString();
         }
-        return JSON_PROVIDER.toJson(jsonPathResult);
+        return JsonUtils.JSON_PROVIDER.toJson(jsonPathResult);
     }
 
     private static boolean isScalar(Object obj) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59ad1948/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
deleted file mode 100644
index ecd2ad0..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.standard.util;
-
-import com.jayway.jsonpath.InvalidPathException;
-import com.jayway.jsonpath.JsonPath;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-
-/**
- * Provides utilities for interacting with JsonPath expressions and results
- *
- * @see <a href="https://github.com/jayway/JsonPath">https://github.com/jayway/JsonPath</a>
- */
-public class JsonPathUtils {
-
-    public static final Validator JSON_PATH_VALIDATOR = new Validator() {
-        @Override
-        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
-            String error = null;
-            try {
-                JsonPath compile = JsonPath.compile(input);
-            } catch (InvalidPathException ipe) {
-                error = ipe.toString();
-            }
-            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
-        }
-    };
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59ad1948/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
new file mode 100644
index 0000000..6eb567e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -0,0 +1,104 @@
+/*
+ * 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.standard.util;
+
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.InvalidPathException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.json.JsonProvider;
+import net.minidev.json.JSONValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.util.BooleanHolder;
+import org.apache.nifi.util.ObjectHolder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+/**
+ * Provides utilities for interacting with JSON elements and JsonPath expressions and results
+ *
+ * @see <a href="http://json.org">http://json.org</a>
+ * @see <a href="https://github.com/jayway/JsonPath">https://github.com/jayway/JsonPath</a>
+ */
+public class JsonUtils {
+
+    public static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider();
+
+    public static final Validator JSON_PATH_VALIDATOR = new Validator() {
+        @Override
+        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+            String error = null;
+            try {
+                JsonPath compile = JsonPath.compile(input);
+            } catch (InvalidPathException ipe) {
+                error = ipe.toString();
+            }
+            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
+        }
+    };
+
+    public static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) {
+
+        final BooleanHolder validJsonHolder = new BooleanHolder(false);
+        processSession.read(flowFile, new InputStreamCallback() {
+            @Override
+            public void process(InputStream in) throws IOException {
+                validJsonHolder.set(JsonUtils.isValidJson(in));
+            }
+        });
+
+        // Parse the document once into an associated context to support multiple path evaluations if specified
+        final ObjectHolder<DocumentContext> contextHolder = new ObjectHolder<>(null);
+
+        if (validJsonHolder.get()) {
+            processSession.read(flowFile, new InputStreamCallback() {
+                @Override
+                public void process(InputStream in) throws IOException {
+                    try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) {
+                        DocumentContext ctx = JsonPath.parse(in);
+                        contextHolder.set(ctx);
+                    }
+                }
+            });
+        }
+
+        return contextHolder.get();
+    }
+
+
+    public static boolean isValidJson(InputStream inputStream) throws IOException {
+        boolean isValid = false;
+
+        /*
+         * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
+         * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
+         */
+        try (InputStreamReader inputStreamReader = new InputStreamReader(inputStream)) {
+            isValid = JSONValue.isValidJsonStrict(inputStreamReader);
+        }
+
+        return isValid;
+    }
+}


[06/29] incubator-nifi git commit: Adjusting filename of EvaluateJsonPath

Posted by jo...@apache.org.
Adjusting filename of EvaluateJsonPath


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/40da65f1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/40da65f1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/40da65f1

Branch: refs/heads/NIFI-360
Commit: 40da65f1937089ea3935f21da384f4bbbc36d20d
Parents: da6b55f
Author: Aldrin Piri <al...@gmail.com>
Authored: Sun Feb 15 15:25:22 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Sun Feb 15 15:25:22 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJSONPath.java   | 152 -------------------
 .../processors/standard/EvaluateJsonPath.java   | 152 +++++++++++++++++++
 2 files changed, 152 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/40da65f1/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
deleted file mode 100644
index 4e1c6ba..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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.standard;
-
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.InvalidPathException;
-import com.jayway.jsonpath.JsonPath;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.*;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.io.InputStreamCallback;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-
-@EventDriven
-@SideEffectFree
-@SupportsBatching
-@Tags({"JSON", "evaluate", "JsonPath"})
-@CapabilityDescription("")
-public class EvaluateJsonPath extends AbstractProcessor {
-
-    public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
-    public static final String DESTINATION_CONTENT = "flowfile-content";
-
-    public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
-            .name("Destination")
-            .description("Indicates whether the results of the JsonPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JsonPath may be specified, and the property name is ignored.")
-            .required(true)
-            .allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTE)
-            .defaultValue(DESTINATION_CONTENT)
-            .build();
-
-    public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
-    public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JsonPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JsonPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
-
-    private Set<Relationship> relationships;
-    private List<PropertyDescriptor> properties;
-
-
-    @Override
-    protected void init(final ProcessorInitializationContext context) {
-        final Set<Relationship> relationships = new HashSet<>();
-        relationships.add(REL_MATCH);
-        relationships.add(REL_NO_MATCH);
-        relationships.add(REL_FAILURE);
-        this.relationships = Collections.unmodifiableSet(relationships);
-
-        final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(DESTINATION);
-        this.properties = Collections.unmodifiableList(properties);
-    }
-
-    @Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
-        final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
-
-        final String destination = context.getProperty(DESTINATION).getValue();
-        if (DESTINATION_CONTENT.equals(destination)) {
-            int jsonPathCount = 0;
-
-            for (final PropertyDescriptor desc : context.getProperties().keySet()) {
-                if (desc.isDynamic()) {
-                    jsonPathCount++;
-                }
-            }
-
-            if (jsonPathCount != 1) {
-                results.add(new ValidationResult.Builder().subject("JsonPaths").valid(false).explanation("Exactly one JsonPath must be set if using destination of " + DESTINATION_CONTENT).build());
-            }
-        }
-
-        return results;
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return properties;
-    }
-
-
-    @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
-        return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName)
-                .expressionLanguageSupported(false)
-                .addValidator(new JsonPathValidator())
-                .required(false)
-                .dynamic(true)
-                .build();
-    }
-
-    @Override
-    public void onTrigger(ProcessContext processContext, ProcessSession processSession) throws ProcessException {
-
-        final FlowFile flowFile = processSession.get();
-        if (flowFile == null) {
-            return;
-        }
-        processSession.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(InputStream in) throws IOException {
-                // Parse the document once to support multiple path evaluations if specified
-                Object document = Configuration.defaultConfiguration().jsonProvider().parse(in, StandardCharsets.UTF_8.displayName());
-            }
-        });
-    }
-
-    private static class JsonPathValidator implements Validator {
-        @Override
-        public ValidationResult validate(String subject, String input, ValidationContext context) {
-            String error = null;
-            try {
-                JsonPath compile = JsonPath.compile(input);
-            } catch (InvalidPathException ipe) {
-                error = ipe.toString();
-            }
-            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/40da65f1/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
new file mode 100644
index 0000000..4e1c6ba
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -0,0 +1,152 @@
+/*
+ * 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.standard;
+
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.InvalidPathException;
+import com.jayway.jsonpath.JsonPath;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.*;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"JSON", "evaluate", "JsonPath"})
+@CapabilityDescription("")
+public class EvaluateJsonPath extends AbstractProcessor {
+
+    public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
+    public static final String DESTINATION_CONTENT = "flowfile-content";
+
+    public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
+            .name("Destination")
+            .description("Indicates whether the results of the JsonPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JsonPath may be specified, and the property name is ignored.")
+            .required(true)
+            .allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTE)
+            .defaultValue(DESTINATION_CONTENT)
+            .build();
+
+    public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
+    public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JsonPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JsonPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
+
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> properties;
+
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_MATCH);
+        relationships.add(REL_NO_MATCH);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DESTINATION);
+        this.properties = Collections.unmodifiableList(properties);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
+
+        final String destination = context.getProperty(DESTINATION).getValue();
+        if (DESTINATION_CONTENT.equals(destination)) {
+            int jsonPathCount = 0;
+
+            for (final PropertyDescriptor desc : context.getProperties().keySet()) {
+                if (desc.isDynamic()) {
+                    jsonPathCount++;
+                }
+            }
+
+            if (jsonPathCount != 1) {
+                results.add(new ValidationResult.Builder().subject("JsonPaths").valid(false).explanation("Exactly one JsonPath must be set if using destination of " + DESTINATION_CONTENT).build());
+            }
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(false)
+                .addValidator(new JsonPathValidator())
+                .required(false)
+                .dynamic(true)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession) throws ProcessException {
+
+        final FlowFile flowFile = processSession.get();
+        if (flowFile == null) {
+            return;
+        }
+        processSession.read(flowFile, new InputStreamCallback() {
+            @Override
+            public void process(InputStream in) throws IOException {
+                // Parse the document once to support multiple path evaluations if specified
+                Object document = Configuration.defaultConfiguration().jsonProvider().parse(in, StandardCharsets.UTF_8.displayName());
+            }
+        });
+    }
+
+    private static class JsonPathValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            String error = null;
+            try {
+                JsonPath compile = JsonPath.compile(input);
+            } catch (InvalidPathException ipe) {
+                error = ipe.toString();
+            }
+            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
+        }
+    }
+}


[17/29] incubator-nifi git commit: Moving the JsonPath Validator to the JsonPathUtil class so that it can be reused by other processors.

Posted by jo...@apache.org.
Moving the JsonPath Validator to the JsonPathUtil class so that it can be reused by other processors.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5a81f19b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5a81f19b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5a81f19b

Branch: refs/heads/NIFI-360
Commit: 5a81f19b25bc073ec1089c115b6e6fcf01056f91
Parents: 0a19ada
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 09:44:30 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 09:44:30 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 21 +++-------
 .../processors/standard/util/JsonPathUtils.java | 44 ++++++++++++++++++++
 2 files changed, 50 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5a81f19b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index fe89635..05d31da 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -16,7 +16,10 @@
  */
 package org.apache.nifi.processors.standard;
 
-import com.jayway.jsonpath.*;
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.PathNotFoundException;
 import com.jayway.jsonpath.spi.json.JsonProvider;
 import net.minidev.json.JSONValue;
 import org.apache.nifi.annotation.behavior.EventDriven;
@@ -27,13 +30,13 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processors.standard.util.JsonPathUtils;
 import org.apache.nifi.stream.io.BufferedInputStream;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.util.BooleanHolder;
@@ -147,7 +150,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .expressionLanguageSupported(false)
-                .addValidator(new JsonPathValidator())
+                .addValidator(JsonPathUtils.JSON_PATH_VALIDATOR)
                 .required(false)
                 .dynamic(true)
                 .build();
@@ -278,17 +281,5 @@ public class EvaluateJsonPath extends AbstractProcessor {
         return (obj instanceof String);
     }
 
-    private static class JsonPathValidator implements Validator {
-        @Override
-        public ValidationResult validate(String subject, String input, ValidationContext context) {
-            String error = null;
-            try {
-                JsonPath compile = JsonPath.compile(input);
-            } catch (InvalidPathException ipe) {
-                error = ipe.toString();
-            }
-            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
-        }
-    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5a81f19b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
new file mode 100644
index 0000000..ecd2ad0
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java
@@ -0,0 +1,44 @@
+/*
+ * 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.standard.util;
+
+import com.jayway.jsonpath.InvalidPathException;
+import com.jayway.jsonpath.JsonPath;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+
+/**
+ * Provides utilities for interacting with JsonPath expressions and results
+ *
+ * @see <a href="https://github.com/jayway/JsonPath">https://github.com/jayway/JsonPath</a>
+ */
+public class JsonPathUtils {
+
+    public static final Validator JSON_PATH_VALIDATOR = new Validator() {
+        @Override
+        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+            String error = null;
+            try {
+                JsonPath compile = JsonPath.compile(input);
+            } catch (InvalidPathException ipe) {
+                error = ipe.toString();
+            }
+            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
+        }
+    };
+}


[10/29] incubator-nifi git commit: Adding a test for multiple attribute paths

Posted by jo...@apache.org.
Adding a test for multiple attribute paths


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/78ad0a31
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/78ad0a31
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/78ad0a31

Branch: refs/heads/NIFI-360
Commit: 78ad0a3147cf1f3573f94f79685ebb868b5d190d
Parents: 974617d
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:20:31 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:20:31 2015 -0500

----------------------------------------------------------------------
 .../standard/TestEvaluateJsonPath.java          | 22 ++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/78ad0a31/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index 308440f..60e19d9 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -96,6 +96,28 @@ public class TestEvaluateJsonPath {
     }
 
     @Test
+    public void testExtractPath_destinationAttributes_twoPaths() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+
+        String jsonPathIdAttrKey = "evaluatejson.id";
+        String jsonPathNameAttrKey = "evaluatejson.name";
+
+        testRunner.setProperty(jsonPathIdAttrKey, "$[0]._id");
+        testRunner.setProperty(jsonPathNameAttrKey, "$[0].name");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
+        Assert.assertEquals("Transferred flow file did not have the correct result for id attribute", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathIdAttrKey));
+        Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", "{\"first\":\"Shaffer\",\"last\":\"Pearson\"}", out.getAttribute(jsonPathNameAttrKey));
+    }
+
+    @Test
     public void testExtractPath_destinationContent() throws Exception {
         String jsonPathAttrKey = "JsonPath";
 


[23/29] incubator-nifi git commit: Adding license terms for asm dependency introduced via JsonPath -> Json-Smart

Posted by jo...@apache.org.
Adding license terms for asm dependency introduced via JsonPath -> Json-Smart


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a058fd04
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a058fd04
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a058fd04

Branch: refs/heads/NIFI-360
Commit: a058fd0460dd5ca91560b1239dd733da37cfa782
Parents: 627bd91
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 16:58:04 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 16:58:04 2015 -0500

----------------------------------------------------------------------
 nifi/LICENSE | 28 ++++++++++++++++++++++++++++
 1 file changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a058fd04/nifi/LICENSE
----------------------------------------------------------------------
diff --git a/nifi/LICENSE b/nifi/LICENSE
index 2e67c09..4d8be01 100644
--- a/nifi/LICENSE
+++ b/nifi/LICENSE
@@ -455,4 +455,32 @@ This product bundles 'json2.js' which is available in the 'public domain'.
 This product bundles 'reset.css' which is available in the 'public domain'.
     For details see http://meyerweb.com/eric/tools/css/reset/
 
+This product bundles 'asm' which is available under an MIT style license.
+For details see http://asm.ow2.org/asmdex-license.html
 
+    Copyright (c) 2012 France Télécom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file


[22/29] incubator-nifi git commit: Adding an auto return type for EvaluateJsonPath to match the semantics of the EvaluateXPath processor.

Posted by jo...@apache.org.
Adding an auto return type for EvaluateJsonPath to match the semantics of the EvaluateXPath processor.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/627bd91f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/627bd91f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/627bd91f

Branch: refs/heads/NIFI-360
Commit: 627bd91fa719748b850b6c610a89b44be3e6f35e
Parents: 1a74621
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 16:15:33 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 16:15:33 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/processors/standard/EvaluateJsonPath.java | 10 +++++++---
 .../nifi/processors/standard/TestEvaluateJsonPath.java    |  1 +
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/627bd91f/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 8cdf1a2..093454d 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -61,6 +61,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
     public static final String DESTINATION_CONTENT = "flowfile-content";
 
+    public static final String RETURN_TYPE_AUTO = "auto-detect";
     public static final String RETURN_TYPE_JSON = "json";
     public static final String RETURN_TYPE_SCALAR = "scalar";
 
@@ -76,8 +77,8 @@ public class EvaluateJsonPath extends AbstractProcessor {
             .name("Return Type")
             .description("Indicates the desired return type of the JSON Path expressions.  Selecting 'auto-detect' will set the return type to 'json' for a Destination of 'flowfile-content', and 'string' for a Destination of 'flowfile-attribute'.")
             .required(true)
-            .allowableValues(RETURN_TYPE_JSON, RETURN_TYPE_SCALAR)
-            .defaultValue(RETURN_TYPE_JSON)
+            .allowableValues(RETURN_TYPE_AUTO, RETURN_TYPE_JSON, RETURN_TYPE_SCALAR)
+            .defaultValue(RETURN_TYPE_AUTO)
             .build();
 
     public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
@@ -168,7 +169,10 @@ public class EvaluateJsonPath extends AbstractProcessor {
         }
 
         final String destination = processContext.getProperty(DESTINATION).getValue();
-        final String returnType = processContext.getProperty(RETURN_TYPE).getValue();
+        String returnType = processContext.getProperty(RETURN_TYPE).getValue();
+        if (returnType.equals(RETURN_TYPE_AUTO)) {
+            returnType = destination.equals(DESTINATION_CONTENT) ? RETURN_TYPE_JSON : RETURN_TYPE_SCALAR;
+        }
 
         flowFileLoop:
         for (FlowFile flowFile : flowFiles) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/627bd91f/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index b7b5103..c5ff814 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -100,6 +100,7 @@ public class TestEvaluateJsonPath {
     public void testExtractPath_destinationAttributes_twoPaths() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
         testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+        testRunner.setProperty(EvaluateJsonPath.RETURN_TYPE, EvaluateJsonPath.RETURN_TYPE_JSON);
 
         String jsonPathIdAttrKey = "evaluatejson.id";
         String jsonPathNameAttrKey = "evaluatejson.name";


[12/29] incubator-nifi git commit: Adding a test for a destination of attributes and only one JsonPath expression evaluates to a found result.

Posted by jo...@apache.org.
Adding a test for a destination of attributes and only one JsonPath expression evaluates to a found result.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/68970907
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/68970907
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/68970907

Branch: refs/heads/NIFI-360
Commit: 6897090771434fc1553b722b835c3b67e8dd14a9
Parents: d4a94c3
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:26:35 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:26:35 2015 -0500

----------------------------------------------------------------------
 .../standard/TestEvaluateJsonPath.java          | 22 ++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/68970907/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index 6a1fbad..fc125c2 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -140,6 +140,28 @@ public class TestEvaluateJsonPath {
     }
 
     @Test
+    public void testExtractPath_destinationAttributes_twoPaths_oneFound() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+
+        String jsonPathIdAttrKey = "evaluatejson.id";
+        String jsonPathNameAttrKey = "evaluatejson.name";
+
+        testRunner.setProperty(jsonPathIdAttrKey, "$[0]._id");
+        testRunner.setProperty(jsonPathNameAttrKey, "$[0].name.nonexistent");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
+        Assert.assertEquals("Transferred flow file did not have the correct result for id attribute", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathIdAttrKey));
+        Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", "", out.getAttribute(jsonPathNameAttrKey));
+    }
+
+    @Test
     public void testExtractPath_destinationContent() throws Exception {
         String jsonPathAttrKey = "JsonPath";
 


[15/29] incubator-nifi git commit: Refining logic of how errors are handled on a per destination basis. Adding supporting tests to ensure contract is met.

Posted by jo...@apache.org.
Refining logic of how errors are handled on a per destination basis.  Adding supporting tests to ensure contract is met.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/bcebba66
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/bcebba66
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/bcebba66

Branch: refs/heads/NIFI-360
Commit: bcebba6632165b6d69eb88ad12895e7678079267
Parents: 5b145e1
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 22:44:06 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 22:44:06 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 62 ++++++++++++++------
 .../standard/TestEvaluateJsonPath.java          | 20 ++++++-
 2 files changed, 61 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bcebba66/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index d6bcdc1..b82764d 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -50,15 +50,23 @@ import java.util.*;
 @SideEffectFree
 @SupportsBatching
 @Tags({"JSON", "evaluate", "JsonPath"})
-@CapabilityDescription("")
+@CapabilityDescription("Evaluates one or more JsonPath expressions against the content of a FlowFile. The results of those expressions are assigned to "
+        + "FlowFile Attributes or are written to the content of the FlowFile itself, depending on configuration of the "
+        + "Processor. JsonPaths are entered by adding user-defined properties; the name of the property maps to the Attribute "
+        + "Name into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the property name is ignored). "
+        + "The value of the property must be a valid JsonPath expression. If the JsonPath evaluates to a JSON array or JSON object and the Return Type is "
+        + "set to 'scalar' the FlowFile will be unmodified and will be routed to failure. If the JsonPath does not "
+        + "evaluate to a scalar, the FlowFile will be routed to 'unmatched' without having its contents modified. If Destination is "
+        + "flowfile-attribute and the expression matches nothing, attributes will be created with empty strings as the value, and the "
+        + "FlowFile will always be routed to 'matched.'  If Destination is 'flowfile-content' and the expression matches nothing, "
+        + "the FlowFile will be routed to 'unmatched' without having its contents modified.")
 public class EvaluateJsonPath extends AbstractProcessor {
 
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
     public static final String DESTINATION_CONTENT = "flowfile-content";
 
-    public static final String RETURN_TYPE_AUTO = "auto-detect";
     public static final String RETURN_TYPE_JSON = "json";
-    public static final String RETURN_TYPE_STRING = "string";
+    public static final String RETURN_TYPE_SCALAR = "scalar";
 
     public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
             .name("Destination")
@@ -72,8 +80,8 @@ public class EvaluateJsonPath extends AbstractProcessor {
             .name("Return Type")
             .description("Indicates the desired return type of the JSON Path expressions.  Selecting 'auto-detect' will set the return type to 'json' for a Destination of 'flowfile-content', and 'string' for a Destination of 'flowfile-attribute'.")
             .required(true)
-            .allowableValues(RETURN_TYPE_AUTO, RETURN_TYPE_JSON, RETURN_TYPE_STRING)
-            .defaultValue(RETURN_TYPE_AUTO)
+            .allowableValues(RETURN_TYPE_JSON, RETURN_TYPE_SCALAR)
+            .defaultValue(RETURN_TYPE_JSON)
             .build();
 
     public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
@@ -208,30 +216,44 @@ public class EvaluateJsonPath extends AbstractProcessor {
             final Map<String, String> jsonPathResults = new HashMap<>();
 
             // Iterate through all JsonPath entries specified
+            jsonPathEvalLoop:
             for (final Map.Entry<String, JsonPath> attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) {
 
                 String jsonPathAttrKey = attributeJsonPathEntry.getKey();
                 JsonPath jsonPathExp = attributeJsonPathEntry.getValue();
 
 
-                final ObjectHolder<String> resultHolder = new ObjectHolder<>("");
+                final ObjectHolder<Object> resultHolder = new ObjectHolder<>(null);
                 try {
-                    resultHolder.set(evaluatePathForContext(jsonPathExp, documentContext));
+                    Object result = documentContext.read(jsonPathExp);
+                    if (RETURN_TYPE.getName().equals(RETURN_TYPE_SCALAR) && !isScalar(result)) {
+                        logger.error("Unable to return a scalar value for a JsonPath {} for FlowFile {}. Transferring to {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), REL_FAILURE.getName()});
+                        processSession.transfer(flowFile, REL_FAILURE);
+                        continue flowFileLoop;
+                    }
+                    resultHolder.set(result);
                 } catch (PathNotFoundException e) {
-                    logger.error("FlowFile {} could not find path {} for attribute key {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), jsonPathAttrKey}, e);
-                    jsonPathResults.put(jsonPathAttrKey, "");
+                    logger.warn("FlowFile {} could not find path {} for attribute key {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), jsonPathAttrKey}, e);
+                    if (destination.equals(DESTINATION_ATTRIBUTE)) {
+                        jsonPathResults.put(jsonPathAttrKey, "");
+                        continue jsonPathEvalLoop;
+                    } else {
+                        processSession.transfer(flowFile, REL_NO_MATCH);
+                        continue flowFileLoop;
+                    }
                 }
 
+                final String resultRepresentation = getResultRepresentation(resultHolder.get());
                 switch (destination) {
                     case DESTINATION_ATTRIBUTE:
-                        jsonPathResults.put(jsonPathAttrKey, resultHolder.get());
+                        jsonPathResults.put(jsonPathAttrKey, resultRepresentation);
                         break;
                     case DESTINATION_CONTENT:
                         flowFile = processSession.write(flowFile, new OutputStreamCallback() {
                             @Override
                             public void process(final OutputStream out) throws IOException {
                                 try (OutputStream outputStream = new BufferedOutputStream(out)) {
-                                    outputStream.write(resultHolder.get().getBytes(StandardCharsets.UTF_8));
+                                    outputStream.write(resultRepresentation.getBytes(StandardCharsets.UTF_8));
                                 }
                             }
                         });
@@ -243,16 +265,18 @@ public class EvaluateJsonPath extends AbstractProcessor {
         }
     }
 
-    private static String evaluatePathForContext(JsonPath path, ReadContext readCtx) {
-        Object pathResult = readCtx.read(path);
+    private static String getResultRepresentation(Object jsonPathResult) {
+        if (isScalar(jsonPathResult)) {
+            return jsonPathResult.toString();
+        }
+        return JSON_PROVIDER.toJson(jsonPathResult);
+    }
+
+    private static boolean isScalar(Object obj) {
         /*
-         *  A given path could be a JSON object or a single value, if a sole value, treat as a String; otherwise, return the
-         *  representative JSON.
+         *  A given path could be a JSON object or a single/scalar value
          */
-        if (pathResult instanceof String) {
-            return pathResult.toString();
-        }
-        return JSON_PROVIDER.toJson(pathResult);
+        return (obj instanceof String);
     }
 
     private static class JsonPathValidator implements Validator {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bcebba66/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index bece3da..e72d069 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -180,7 +180,7 @@ public class TestEvaluateJsonPath {
 
 
     @Test
-    public void testExtractPath_destinationAttribute_indefiniteResult() throws Exception {
+    public void testExtractPath_destinationContent_indefiniteResult() throws Exception {
         String jsonPathAttrKey = "friends.indefinite.id.list";
 
         final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
@@ -197,7 +197,7 @@ public class TestEvaluateJsonPath {
     }
 
     @Test
-    public void testExtractPath_destinationAttribute_indefiniteResult_operators() throws Exception {
+    public void testExtractPath_destinationContent_indefiniteResult_operators() throws Exception {
         String jsonPathAttrKey = "friends.indefinite.id.list";
 
         final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
@@ -212,4 +212,20 @@ public class TestEvaluateJsonPath {
         testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
         testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals("[0,1,2]");
     }
+
+    @Test
+    public void testRouteUnmatched_destinationContent_noMatch() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty("jsonPath", "$[0].nonexistent.key");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_NO_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals(JSON_SNIPPET);
+    }
+
 }


[08/29] incubator-nifi git commit: Completing initial functionality of EvaluateJsonPath and associated tests.

Posted by jo...@apache.org.
Completing initial functionality of EvaluateJsonPath and associated tests.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b3328490
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b3328490
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b3328490

Branch: refs/heads/NIFI-360
Commit: b3328490c65d0b08bed4f711b6990d10262bbef2
Parents: c3c4d36
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:03:24 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:03:24 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 143 ++++++++++++++-----
 .../standard/TestEvaluateJsonPath.java          |  44 ++++++
 2 files changed, 153 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b3328490/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 41da277..e747838 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -29,14 +29,21 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.util.BooleanHolder;
 import org.apache.nifi.util.ObjectHolder;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.*;
 
 @EventDriven
@@ -49,6 +56,10 @@ public class EvaluateJsonPath extends AbstractProcessor {
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
     public static final String DESTINATION_CONTENT = "flowfile-content";
 
+    public static final String RETURN_TYPE_AUTO = "auto-detect";
+    public static final String RETURN_TYPE_JSON = "json";
+    public static final String RETURN_TYPE_STRING = "string";
+
     public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
             .name("Destination")
             .description("Indicates whether the results of the JsonPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JsonPath may be specified, and the property name is ignored.")
@@ -57,6 +68,14 @@ public class EvaluateJsonPath extends AbstractProcessor {
             .defaultValue(DESTINATION_CONTENT)
             .build();
 
+    public static final PropertyDescriptor RETURN_TYPE = new PropertyDescriptor.Builder()
+            .name("Return Type")
+            .description("Indicates the desired return type of the JSON Path expressions.  Selecting 'auto-detect' will set the return type to 'json' for a Destination of 'flowfile-content', and 'string' for a Destination of 'flowfile-attribute'.")
+            .required(true)
+            .allowableValues(RETURN_TYPE_AUTO, RETURN_TYPE_AUTO, RETURN_TYPE_STRING)
+            .defaultValue(RETURN_TYPE_AUTO)
+            .build();
+
     public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JsonPath is successfully evaluated and the FlowFile is modified as a result").build();
     public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JsonPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
     public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JsonPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
@@ -77,6 +96,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
 
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(DESTINATION);
+        properties.add(RETURN_TYPE);
         this.properties = Collections.unmodifiableList(properties);
     }
 
@@ -127,56 +147,111 @@ public class EvaluateJsonPath extends AbstractProcessor {
     @Override
     public void onTrigger(ProcessContext processContext, final ProcessSession processSession) throws ProcessException {
 
-        final FlowFile flowFile = processSession.get();
-        if (flowFile == null) {
+        List<FlowFile> flowFiles = processSession.get(50);
+        if (flowFiles.isEmpty()) {
             return;
         }
 
-        // Determine the destination
+        final ProcessorLog logger = getLogger();
+
+        /* Build the JsonPath expressions from attributes */
+        final Map<String, JsonPath> attributeToJsonPathMap = new HashMap<>();
+
+        for (final Map.Entry<PropertyDescriptor, String> entry : processContext.getProperties().entrySet()) {
+            if (!entry.getKey().isDynamic()) {
+                continue;
+            }
+            final JsonPath jsonPath = JsonPath.compile(entry.getValue());
+            attributeToJsonPathMap.put(entry.getKey().getName(), jsonPath);
+        }
 
         final String destination = processContext.getProperty(DESTINATION).getValue();
 
-        final ObjectHolder<DocumentContext> contextHolder = new ObjectHolder<>(null);
-
-        // Parse the document once to support multiple path evaluations if specified
-        processSession.read(flowFile, new InputStreamCallback() {
-            @Override
-            public void process(InputStream in) throws IOException {
-                /*
-                 * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
-                 * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
-                 */
-                boolean validJson = JSONValue.isValidJsonStrict(new InputStreamReader(in));
-                if (validJson) {
-                    DocumentContext ctx = JsonPath.parse(in);
-                    contextHolder.set(ctx);
-                } else {
-                    getLogger().error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile.getId()});
-                    processSession.transfer(flowFile, REL_FAILURE);
+        flowFileLoop:
+        for (FlowFile flowFile : flowFiles) {
+            // Validate the JSON document before attempting processing
+            final BooleanHolder validJsonHolder = new BooleanHolder(false);
+            processSession.read(flowFile, new InputStreamCallback() {
+                @Override
+                public void process(InputStream in) throws IOException {
+                    try (InputStreamReader inputStreamReader = new InputStreamReader(in)) {
+                        /*
+                         * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
+                         * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
+                         */
+                        boolean validJson = JSONValue.isValidJsonStrict(inputStreamReader);
+                        validJsonHolder.set(validJson);
+                    }
                 }
+            });
+
+            if (!validJsonHolder.get()) {
+                logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile.getId()});
+                processSession.transfer(flowFile, REL_FAILURE);
+                continue flowFileLoop;
             }
-        });
 
-        DocumentContext documentContext = contextHolder.get();
+            // Parse the document once into an associated context to support multiple path evaluations if specified
+            final ObjectHolder<DocumentContext> contextHolder = new ObjectHolder<>(null);
+            processSession.read(flowFile, new InputStreamCallback() {
+                @Override
+                public void process(InputStream in) throws IOException {
+                    try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) {
+                        DocumentContext ctx = JsonPath.parse(in);
+                        contextHolder.set(ctx);
+                    }
+                }
+            });
 
-        if (documentContext == null) {
-            return;
-        }
+            final DocumentContext documentContext = contextHolder.get();
+
+            final Map<String, String> jsonPathResults = new HashMap<>();
+
+            // Iterate through all JsonPath entries specified
+            for (final Map.Entry<String, JsonPath> attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) {
+
+                String jsonPathAttrKey = attributeJsonPathEntry.getKey();
+                JsonPath jsonPathExp = attributeJsonPathEntry.getValue();
+                final String evalResult = evaluatePathForContext(jsonPathExp, documentContext);
 
-        try {
-            switch (destination) {
-                case DESTINATION_ATTRIBUTE:
-                    break;
-                case DESTINATION_CONTENT:
-                    break;
+                try {
+                    switch (destination) {
+                        case DESTINATION_ATTRIBUTE:
+                            jsonPathResults.put(jsonPathAttrKey, evalResult);
+                            break;
+                        case DESTINATION_CONTENT:
+                            flowFile = processSession.write(flowFile, new OutputStreamCallback() {
+                                @Override
+                                public void process(final OutputStream out) throws IOException {
+                                    try (OutputStream outputStream = new BufferedOutputStream(out)) {
+                                        outputStream.write(evalResult.getBytes(StandardCharsets.UTF_8));
+                                    }
+                                }
+                            });
+                            break;
+                    }
+                } catch (PathNotFoundException e) {
+                    logger.error("FlowFile {} could not find path {} for attribute key {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), jsonPathAttrKey}, e);
+                    jsonPathResults.put(jsonPathAttrKey, "");
+                }
             }
+            flowFile = processSession.putAllAttributes(flowFile, jsonPathResults);
             processSession.transfer(flowFile, REL_MATCH);
-        } catch (PathNotFoundException e) {
-            getLogger().warn("FlowFile {} could not be read from.", new Object[]{flowFile.getId()}, e);
-            processSession.transfer(flowFile, REL_NO_MATCH);
         }
     }
 
+    private static String evaluatePathForContext(JsonPath path, ReadContext readCtx) {
+        Object pathResult = readCtx.read(path);
+        /*
+         *  A given path could be a JSON object or a single value, if a sole value, treat as a String; otherwise, return the
+         *  representative JSON.
+         */
+        if (pathResult instanceof String) {
+            return pathResult.toString();
+        }
+        return JSON_PROVIDER.toJson(pathResult);
+    }
+
     private static class JsonPathValidator implements Validator {
         @Override
         public ValidationResult validate(String subject, String input, ValidationContext context) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b3328490/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index a4e65cf..7174e9a 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.standard;
 
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -50,4 +51,47 @@ public class TestEvaluateJsonPath {
         testRunner.assertAllFlowFilesTransferred(EvaluateJsonPath.REL_FAILURE, 1);
         final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateJsonPath.REL_FAILURE).get(0);
     }
+
+
+    @Test(expected = AssertionError.class)
+    public void testInvalidConfiguration_destinationContent_twoPaths() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty("JsonPath1", "$[0]._id");
+        testRunner.setProperty("JsonPath2", "$[0].name");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Assert.fail("Processor incorrectly ran with an invalid configuration of multiple paths specified as attributes for a destination of content.");
+    }
+
+    @Test
+    public void testConfiguration_destinationAttributes_twoPaths() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+        testRunner.setProperty("JsonPath1", "$[0]._id");
+        testRunner.setProperty("JsonPath2", "$[0].name");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+    }
+
+    @Test
+    public void testExtractPath_destinationAttribute() throws Exception {
+        String jsonPathAttrKey = "JsonPath";
+
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+        testRunner.setProperty(jsonPathAttrKey, "$[0]._id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
+        Assert.assertEquals("Transferred flow file did not have the correct result", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathAttrKey));
+    }
 }


[19/29] incubator-nifi git commit: Providing a SplitJson processor which will break JSON Arrays into their individual elements. Refactored supporting JsonUtils code and EvaluateJsonPath to reuse common functionality.

Posted by jo...@apache.org.
Providing a SplitJson processor which will break JSON Arrays into their individual elements.  Refactored supporting JsonUtils code and EvaluateJsonPath to reuse common functionality.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/2e05dcbb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/2e05dcbb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/2e05dcbb

Branch: refs/heads/NIFI-360
Commit: 2e05dcbbfdb37843456dff121d9878d8679d7067
Parents: 59ad194
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 15:15:53 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 15:17:12 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   |   8 +-
 .../nifi/processors/standard/SplitJson.java     | 140 +++++++++++++++++++
 .../processors/standard/util/JsonUtils.java     |   8 ++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../standard/TestEvaluateJsonPath.java          |   3 +-
 .../nifi/processors/standard/TestSplitJson.java | 115 +++++++++++++++
 6 files changed, 268 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 0bdd20c..8cdf1a2 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -192,7 +192,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
                 final ObjectHolder<Object> resultHolder = new ObjectHolder<>(null);
                 try {
                     Object result = documentContext.read(jsonPathExp);
-                    if (returnType.equals(RETURN_TYPE_SCALAR) && !isScalar(result)) {
+                    if (returnType.equals(RETURN_TYPE_SCALAR) && !JsonUtils.isJsonScalar(result)) {
                         logger.error("Unable to return a scalar value for the expression {} for FlowFile {}. Evaluated value was {}. Transferring to {}.",
                                 new Object[]{jsonPathExp.getPath(), flowFile.getId(), result.toString(), REL_FAILURE.getName()});
                         processSession.transfer(flowFile, REL_FAILURE);
@@ -233,15 +233,11 @@ public class EvaluateJsonPath extends AbstractProcessor {
     }
 
     private static String getResultRepresentation(Object jsonPathResult) {
-        if (isScalar(jsonPathResult)) {
+        if (JsonUtils.isJsonScalar(jsonPathResult)) {
             return jsonPathResult.toString();
         }
         return JsonUtils.JSON_PROVIDER.toJson(jsonPathResult);
     }
 
-    private static boolean isScalar(Object obj) {
-        return (obj instanceof String);
-    }
-
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
new file mode 100644
index 0000000..e589b48
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
@@ -0,0 +1,140 @@
+/*
+ * 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.standard;
+
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ProcessorLog;
+import org.apache.nifi.processor.*;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processors.standard.util.JsonUtils;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"json", "split", "jsonpath"})
+@CapabilityDescription("Splits a JSON File into multiple, separate FlowFiles for an array element specified by a JsonPath expression. "
+        + "Each generated FlowFile is comprised of an element of the specified array and transferred to relationship 'split, "
+        + "with the original file transferred to the 'original' relationship. If the specified JsonPath is not found or "
+        + "does not evaluate to an array element, the original file is routed to 'failure' and no files are generated.")
+public class SplitJson extends AbstractProcessor {
+
+    public static final PropertyDescriptor ARRAY_JSON_PATH_EXPRESSION = new PropertyDescriptor.Builder()
+            .name("JsonPath Expression")
+            .description("A JsonPath expression that indicates the array element to split into JSON/scalar fragments.")
+            .required(true)
+            .addValidator(JsonUtils.JSON_PATH_VALIDATOR)
+            .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder().name("original").description("The original FlowFile that was split into segments. If the FlowFile fails processing, nothing will be sent to this relationship").build();
+    public static final Relationship REL_SPLIT = new Relationship.Builder().name("split").description("All segments of the original FlowFile will be routed to this relationship").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("If a FlowFile fails processing for any reason (for example, the FlowFile is not valid JSON or the specified path does not exist), it will be routed to this relationship").build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(ARRAY_JSON_PATH_EXPRESSION);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_SPLIT);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext processContext, final ProcessSession processSession) {
+        final FlowFile original = processSession.get();
+        if (original == null) {
+            return;
+        }
+
+        final ProcessorLog logger = getLogger();
+
+
+        final DocumentContext documentContext = JsonUtils.validateAndEstablishJsonContext(processSession, original);
+
+        if (documentContext == null) {
+            logger.error("FlowFile {} did not have valid JSON content.", new Object[]{original});
+            processSession.transfer(original, REL_FAILURE);
+            return;
+        }
+
+        final String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue();
+        final JsonPath jsonPath = JsonPath.compile(jsonPathExpression);
+
+        final List<FlowFile> segments = new ArrayList<>();
+
+        Object jsonPathResult = documentContext.read(jsonPath);
+
+        if (!(jsonPathResult instanceof List)) {
+            logger.error("The evaluated value {} of {} was not an array compatible type and cannot be split.",
+                    new Object[]{jsonPathResult, jsonPath.getPath()});
+            processSession.transfer(original, REL_FAILURE);
+            return;
+        }
+
+        List resultList = (List) jsonPathResult;
+
+        for (final Object resultSegment : resultList) {
+            FlowFile split = processSession.create(original);
+            split = processSession.write(split, new OutputStreamCallback() {
+                @Override
+                public void process(OutputStream out) throws IOException {
+                    String resultSegmentContent;
+                    if (JsonUtils.isJsonScalar(resultSegment)) {
+                        resultSegmentContent = resultSegment.toString();
+                    } else {
+                        resultSegmentContent = JsonUtils.JSON_PROVIDER.toJson(resultSegment);
+                    }
+                    out.write(resultSegmentContent.getBytes(StandardCharsets.UTF_8));
+                }
+            });
+            segments.add(split);
+        }
+
+        processSession.transfer(segments, REL_SPLIT);
+        processSession.transfer(original, REL_ORIGINAL);
+        logger.info("Split {} into {} FlowFiles", new Object[]{original, segments.size()});
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
index 6eb567e..0bf33dd 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -35,6 +35,8 @@ import org.apache.nifi.util.ObjectHolder;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Provides utilities for interacting with JSON elements and JsonPath expressions and results
@@ -101,4 +103,10 @@ public class JsonUtils {
 
         return isValid;
     }
+
+    public static boolean isJsonScalar(Object obj) {
+        // For the default provider, a Map or List is able to be handled as a JSON entity
+        return !(obj instanceof Map || obj instanceof List);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 8a1fd74..17b5364 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -55,6 +55,7 @@ org.apache.nifi.processors.standard.ScanAttribute
 org.apache.nifi.processors.standard.ScanContent
 org.apache.nifi.processors.standard.SegmentContent
 org.apache.nifi.processors.standard.SplitContent
+org.apache.nifi.processors.standard.SplitJson
 org.apache.nifi.processors.standard.SplitText
 org.apache.nifi.processors.standard.SplitXml
 org.apache.nifi.processors.standard.TransformXml

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index c873969..b7b5103 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.StringUtils;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
@@ -158,7 +159,7 @@ public class TestEvaluateJsonPath {
         testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
         final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
         Assert.assertEquals("Transferred flow file did not have the correct result for id attribute", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathIdAttrKey));
-        Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", "", out.getAttribute(jsonPathNameAttrKey));
+        Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", StringUtils.EMPTY, out.getAttribute(jsonPathNameAttrKey));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2e05dcbb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
new file mode 100644
index 0000000..dd6fc6d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
@@ -0,0 +1,115 @@
+/*
+ * 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.standard;
+
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class TestSplitJson {
+
+    private static final Path JSON_SNIPPET = Paths.get("src/test/resources/TestJson/json-sample.json");
+    private static final Path XML_SNIPPET = Paths.get("src/test/resources/TestXml/xml-snippet.xml");
+
+    @Test(expected = AssertionError.class)
+    public void testInvalidJsonPath() {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$..");
+
+        Assert.fail("An improper JsonPath expression was not detected as being invalid.");
+    }
+
+    @Test
+    public void testInvalidJsonDocument() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$");
+
+        testRunner.enqueue(XML_SNIPPET);
+        testRunner.run();
+
+        testRunner.assertAllFlowFilesTransferred(SplitJson.REL_FAILURE, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).get(0);
+        // Verify that the content was unchanged
+        out.assertContentEquals(XML_SNIPPET);
+    }
+
+    @Test
+    public void testSplit_nonArrayResult() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$[0]._id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = SplitJson.REL_FAILURE;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
+        out.assertContentEquals(JSON_SNIPPET);
+    }
+
+    @Test
+    public void testSplit_arrayResult_oneValue() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$[0].range[?(@ == 0)]");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1);
+        testRunner.assertTransferCount(SplitJson.REL_SPLIT, 1);
+        testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertContentEquals(JSON_SNIPPET);
+        testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0).assertContentEquals("0");
+    }
+
+    @Test
+    public void testSplit_arrayResult_multipleValues() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$[0].range");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        int numSplitsExpected = 10;
+
+        testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1);
+        testRunner.assertTransferCount(SplitJson.REL_SPLIT, numSplitsExpected);
+        final MockFlowFile originalOut = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0);
+        originalOut.assertContentEquals(JSON_SNIPPET);
+    }
+
+    @Test
+    public void testSplit_arrayResult_nonScalarValues() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$[*].name");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1);
+        testRunner.assertTransferCount(SplitJson.REL_SPLIT, 7);
+        testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertContentEquals(JSON_SNIPPET);
+        testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0).assertContentEquals("{\"first\":\"Shaffer\",\"last\":\"Pearson\"}");
+    }
+
+}


[24/29] incubator-nifi git commit: Preferring the constant empty string for clarity instead of quotes.

Posted by jo...@apache.org.
Preferring the constant empty string for clarity instead of quotes.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/408f6363
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/408f6363
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/408f6363

Branch: refs/heads/NIFI-360
Commit: 408f636361cbad025752978e10775651b7daddf4
Parents: a058fd0
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 17:01:06 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 17:01:06 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/processors/standard/EvaluateJsonPath.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/408f6363/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 093454d..3a64a34 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -35,6 +35,7 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processors.standard.util.JsonUtils;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.util.ObjectHolder;
+import org.apache.nifi.util.StringUtils;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -206,7 +207,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
                 } catch (PathNotFoundException e) {
                     logger.warn("FlowFile {} could not find path {} for attribute key {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), jsonPathAttrKey}, e);
                     if (destination.equals(DESTINATION_ATTRIBUTE)) {
-                        jsonPathResults.put(jsonPathAttrKey, "");
+                        jsonPathResults.put(jsonPathAttrKey, StringUtils.EMPTY);
                         continue jsonPathEvalLoop;
                     } else {
                         processSession.transfer(flowFile, REL_NO_MATCH);


[02/29] incubator-nifi git commit: Creating stub for an EvaluateJSONPath processor and providing configuration akin to EvaluateXPath

Posted by jo...@apache.org.
Creating stub for an EvaluateJSONPath processor and providing configuration akin to EvaluateXPath


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5f03fb11
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5f03fb11
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5f03fb11

Branch: refs/heads/NIFI-360
Commit: 5f03fb11d93791ea9cdde43ffb97bceaf22c0143
Parents: 446fc2e
Author: Aldrin Piri <al...@gmail.com>
Authored: Sat Feb 14 12:56:19 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Sat Feb 14 12:56:19 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJSONPath.java   | 59 ++++++++++++++++++++
 1 file changed, 59 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5f03fb11/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
new file mode 100644
index 0000000..9ad3e49
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
@@ -0,0 +1,59 @@
+/*
+ * 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.standard;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"JSON", "evaluate", "JSONPath"})
+@CapabilityDescription("")
+public class EvaluateJSONPath extends AbstractProcessor {
+
+    public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
+    public static final String DESTINATION_CONTENT = "flowfile-content";
+
+    public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
+            .name("Destination")
+            .description("Indicates whether the results of the JSONPath evaluation are written to the FlowFile content or a FlowFile attribute; if using attribute, must specify the Attribute Name property. If set to flowfile-content, only one JSONPath may be specified, and the property name is ignored.")
+            .required(true)
+            .allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTE)
+            .defaultValue(DESTINATION_CONTENT)
+            .build();
+
+    public static final Relationship REL_MATCH = new Relationship.Builder().name("matched").description("FlowFiles are routed to this relationship when the JSONPath is successfully evaluated and the FlowFile is modified as a result").build();
+    public static final Relationship REL_NO_MATCH = new Relationship.Builder().name("unmatched").description("FlowFiles are routed to this relationship when the JSONPath does not match the content of the FlowFile and the Destination is set to flowfile-content").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles are routed to this relationship when the JSONPath cannot be evaluated against the content of the FlowFile; for instance, if the FlowFile is not valid JSON").build();
+
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession) throws ProcessException {
+
+    }
+
+}


[14/29] incubator-nifi git commit: Adjusting error where the return type of string was omitted from the possible selections

Posted by jo...@apache.org.
Adjusting error where the return type of string was omitted from the possible selections


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5b145e10
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5b145e10
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5b145e10

Branch: refs/heads/NIFI-360
Commit: 5b145e10e8508bec5293aab7318978fc3c9e2b57
Parents: 7e58130
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:34:23 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:34:23 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/processors/standard/EvaluateJsonPath.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5b145e10/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 13eb50b..d6bcdc1 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -72,7 +72,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
             .name("Return Type")
             .description("Indicates the desired return type of the JSON Path expressions.  Selecting 'auto-detect' will set the return type to 'json' for a Destination of 'flowfile-content', and 'string' for a Destination of 'flowfile-attribute'.")
             .required(true)
-            .allowableValues(RETURN_TYPE_AUTO, RETURN_TYPE_AUTO, RETURN_TYPE_STRING)
+            .allowableValues(RETURN_TYPE_AUTO, RETURN_TYPE_JSON, RETURN_TYPE_STRING)
             .defaultValue(RETURN_TYPE_AUTO)
             .build();
 


[29/29] incubator-nifi git commit: NIFI-360 providing review feedback on github PR

Posted by jo...@apache.org.
NIFI-360 providing review feedback on github PR


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/28627712
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/28627712
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/28627712

Branch: refs/heads/NIFI-360
Commit: 2862771235eb7a9e007c1a14096098d9809987be
Parents: f05cc93
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 17 23:29:16 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 17 23:29:16 2015 -0500

----------------------------------------------------------------------
 nifi/LICENSE                                    | 29 --------------------
 nifi/nifi-assembly/LICENSE                      | 28 +++++++++++++++++++
 nifi/nifi-assembly/NOTICE                       |  4 +++
 .../src/main/resources/META-INF/LICENSE         | 29 ++++++++++++++++++++
 .../src/main/resources/META-INF/NOTICE          |  4 +++
 5 files changed, 65 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/28627712/nifi/LICENSE
----------------------------------------------------------------------
diff --git a/nifi/LICENSE b/nifi/LICENSE
index 4d8be01..50150e3 100644
--- a/nifi/LICENSE
+++ b/nifi/LICENSE
@@ -455,32 +455,3 @@ This product bundles 'json2.js' which is available in the 'public domain'.
 This product bundles 'reset.css' which is available in the 'public domain'.
     For details see http://meyerweb.com/eric/tools/css/reset/
 
-This product bundles 'asm' which is available under an MIT style license.
-For details see http://asm.ow2.org/asmdex-license.html
-
-    Copyright (c) 2012 France Télécom
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions
-    are met:
-    1. Redistributions of source code must retain the above copyright
-       notice, this list of conditions and the following disclaimer.
-    2. Redistributions in binary form must reproduce the above copyright
-       notice, this list of conditions and the following disclaimer in the
-       documentation and/or other materials provided with the distribution.
-    3. Neither the name of the copyright holders nor the names of its
-       contributors may be used to endorse or promote products derived from
-       this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
-    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
-    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
-    THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/28627712/nifi/nifi-assembly/LICENSE
----------------------------------------------------------------------
diff --git a/nifi/nifi-assembly/LICENSE b/nifi/nifi-assembly/LICENSE
index da81c41..bf9e649 100644
--- a/nifi/nifi-assembly/LICENSE
+++ b/nifi/nifi-assembly/LICENSE
@@ -799,5 +799,33 @@ This product bundles 'JCraft Jzlib' which is available under a 3-Clause BSD Lice
     NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
     EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
+This product bundles 'asm' which is available under an MIT style license.
+For details see http://asm.ow2.org/asmdex-license.html
 
+    Copyright (c) 2012 France Télécom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/28627712/nifi/nifi-assembly/NOTICE
----------------------------------------------------------------------
diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE
index 7d42538..1667e9e 100644
--- a/nifi/nifi-assembly/NOTICE
+++ b/nifi/nifi-assembly/NOTICE
@@ -462,6 +462,10 @@ The following binary components are provided under the Apache Software License v
       This product includes software developed by the
       Visigoth Software Society (http://www.visigoths.org/).
 
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
+
 ************************
 Common Development and Distribution License 1.1
 ************************

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/28627712/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/LICENSE b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/LICENSE
index 44bb0c0..36ba155 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/LICENSE
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/LICENSE
@@ -290,3 +290,32 @@ This product bundles 'JCraft Jzlib' which is available under a 3-Clause BSD Lice
     NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
     EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
+This product bundles 'asm' which is available under an MIT style license.
+For details see http://asm.ow2.org/asmdex-license.html
+
+    Copyright (c) 2012 France Télécom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/28627712/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
index c5a1533..d63a74d 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE
@@ -78,6 +78,10 @@ The following binary components are provided under the Apache Software License v
           Written by Doug Lea with assistance from members of JCP JSR-166
           Expert Group and released to the public domain, as explained at
           http://creativecommons.org/publicdomain/zero/1.0/
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
       
 ************************
 Common Development and Distribution License 1.1


[16/29] incubator-nifi git commit: Adjusting comments and providing a test to ensure a non-scalar value for a specified scalar return type is routed to failure.

Posted by jo...@apache.org.
Adjusting comments and providing a test to ensure a non-scalar value for a specified scalar return type is routed to failure.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/0a19ada0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/0a19ada0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/0a19ada0

Branch: refs/heads/NIFI-360
Commit: 0a19ada0a3229b1c8a72fe638f286db33b6f7ba6
Parents: bcebba6
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 23:26:45 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 23:26:45 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 31 ++++++++++----------
 .../standard/TestEvaluateJsonPath.java          | 19 ++++++++++++
 2 files changed, 34 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0a19ada0/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index b82764d..fe89635 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -50,16 +50,17 @@ import java.util.*;
 @SideEffectFree
 @SupportsBatching
 @Tags({"JSON", "evaluate", "JsonPath"})
-@CapabilityDescription("Evaluates one or more JsonPath expressions against the content of a FlowFile. The results of those expressions are assigned to "
-        + "FlowFile Attributes or are written to the content of the FlowFile itself, depending on configuration of the "
-        + "Processor. JsonPaths are entered by adding user-defined properties; the name of the property maps to the Attribute "
-        + "Name into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the property name is ignored). "
-        + "The value of the property must be a valid JsonPath expression. If the JsonPath evaluates to a JSON array or JSON object and the Return Type is "
-        + "set to 'scalar' the FlowFile will be unmodified and will be routed to failure. If the JsonPath does not "
-        + "evaluate to a scalar, the FlowFile will be routed to 'unmatched' without having its contents modified. If Destination is "
-        + "flowfile-attribute and the expression matches nothing, attributes will be created with empty strings as the value, and the "
-        + "FlowFile will always be routed to 'matched.'  If Destination is 'flowfile-content' and the expression matches nothing, "
-        + "the FlowFile will be routed to 'unmatched' without having its contents modified.")
+@CapabilityDescription("Evaluates one or more JsonPath expressions against the content of a FlowFile. "
+        + "The results of those expressions are assigned to FlowFile Attributes or are written to the content of the FlowFile itself, "
+        + "depending on configuration of the Processor. "
+        + "JsonPaths are entered by adding user-defined properties; the name of the property maps to the Attribute Name "
+        + "into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the property name is ignored). "
+        + "The value of the property must be a valid JsonPath expression. "
+        + "If the JsonPath evaluates to a JSON array or JSON object and the Return Type is set to 'scalar' the FlowFile will be unmodified and will be routed to failure. "
+        + "A Return Type of JSON can return scalar values if the provided JsonPath evaluates to the specified value and will be routed as a match."
+        + "If Destination is 'flowfile-content' and the JsonPath does not evaluate to a defined path, the FlowFile will be routed to 'unmatched' without having its contents modified. "
+        + "If Destination is flowfile-attribute and the expression matches nothing, attributes will be created with "
+        + "empty strings as the value, and the FlowFile will always be routed to 'matched.'")
 public class EvaluateJsonPath extends AbstractProcessor {
 
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
@@ -174,6 +175,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
         }
 
         final String destination = processContext.getProperty(DESTINATION).getValue();
+        final String returnType = processContext.getProperty(RETURN_TYPE).getValue();
 
         flowFileLoop:
         for (FlowFile flowFile : flowFiles) {
@@ -215,7 +217,6 @@ public class EvaluateJsonPath extends AbstractProcessor {
 
             final Map<String, String> jsonPathResults = new HashMap<>();
 
-            // Iterate through all JsonPath entries specified
             jsonPathEvalLoop:
             for (final Map.Entry<String, JsonPath> attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) {
 
@@ -226,8 +227,9 @@ public class EvaluateJsonPath extends AbstractProcessor {
                 final ObjectHolder<Object> resultHolder = new ObjectHolder<>(null);
                 try {
                     Object result = documentContext.read(jsonPathExp);
-                    if (RETURN_TYPE.getName().equals(RETURN_TYPE_SCALAR) && !isScalar(result)) {
-                        logger.error("Unable to return a scalar value for a JsonPath {} for FlowFile {}. Transferring to {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), REL_FAILURE.getName()});
+                    if (returnType.equals(RETURN_TYPE_SCALAR) && !isScalar(result)) {
+                        logger.error("Unable to return a scalar value for the expression {} for FlowFile {}. Evaluated value was {}. Transferring to {}.",
+                                new Object[]{jsonPathExp.getPath(), flowFile.getId(), result.toString(), REL_FAILURE.getName()});
                         processSession.transfer(flowFile, REL_FAILURE);
                         continue flowFileLoop;
                     }
@@ -273,9 +275,6 @@ public class EvaluateJsonPath extends AbstractProcessor {
     }
 
     private static boolean isScalar(Object obj) {
-        /*
-         *  A given path could be a JSON object or a single/scalar value
-         */
         return (obj instanceof String);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0a19ada0/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index e72d069..c873969 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -228,4 +228,23 @@ public class TestEvaluateJsonPath {
         testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals(JSON_SNIPPET);
     }
 
+
+    @Test
+    public void testRouteFailure_returnTypeScalar_resultArray() throws Exception {
+        String jsonPathAttrKey = "friends.indefinite.id.list";
+
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.RETURN_TYPE, EvaluateJsonPath.RETURN_TYPE_SCALAR);
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty(jsonPathAttrKey, "$[0].friends[?(@.id < 3)].id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_FAILURE;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals(JSON_SNIPPET);
+    }
+
 }


[09/29] incubator-nifi git commit: Adding a test to verify the placement of a JSON path expression with a content destination.

Posted by jo...@apache.org.
Adding a test to verify the placement of a JSON path expression with a content destination.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/974617d4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/974617d4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/974617d4

Branch: refs/heads/NIFI-360
Commit: 974617d44ee41bd5cac9f238eca43cb9f81684a6
Parents: b332849
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:09:22 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:09:22 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/TestEvaluateJsonPath.java  | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/974617d4/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index 7174e9a..308440f 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -94,4 +94,21 @@ public class TestEvaluateJsonPath {
         final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
         Assert.assertEquals("Transferred flow file did not have the correct result", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathAttrKey));
     }
+
+    @Test
+    public void testExtractPath_destinationContent() throws Exception {
+        String jsonPathAttrKey = "JsonPath";
+
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT);
+        testRunner.setProperty(jsonPathAttrKey, "$[0]._id");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        testRunner.getFlowFilesForRelationship(expectedRel).get(0).assertContentEquals("54df94072d5dbf7dc6340cc5");
+    }
 }


[11/29] incubator-nifi git commit: Adding a test for multiple attributes where neither evaluates to a found path.

Posted by jo...@apache.org.
Adding a test for multiple attributes where neither evaluates to a found path.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d4a94c37
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d4a94c37
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d4a94c37

Branch: refs/heads/NIFI-360
Commit: d4a94c37eea9c4a545e9f764c113a3d863257e57
Parents: 78ad0a3
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 18:24:57 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 18:24:57 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 35 +++++++++++---------
 .../standard/TestEvaluateJsonPath.java          | 22 ++++++++++++
 2 files changed, 41 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d4a94c37/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index e747838..13eb50b 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -212,28 +212,31 @@ public class EvaluateJsonPath extends AbstractProcessor {
 
                 String jsonPathAttrKey = attributeJsonPathEntry.getKey();
                 JsonPath jsonPathExp = attributeJsonPathEntry.getValue();
-                final String evalResult = evaluatePathForContext(jsonPathExp, documentContext);
 
+
+                final ObjectHolder<String> resultHolder = new ObjectHolder<>("");
                 try {
-                    switch (destination) {
-                        case DESTINATION_ATTRIBUTE:
-                            jsonPathResults.put(jsonPathAttrKey, evalResult);
-                            break;
-                        case DESTINATION_CONTENT:
-                            flowFile = processSession.write(flowFile, new OutputStreamCallback() {
-                                @Override
-                                public void process(final OutputStream out) throws IOException {
-                                    try (OutputStream outputStream = new BufferedOutputStream(out)) {
-                                        outputStream.write(evalResult.getBytes(StandardCharsets.UTF_8));
-                                    }
-                                }
-                            });
-                            break;
-                    }
+                    resultHolder.set(evaluatePathForContext(jsonPathExp, documentContext));
                 } catch (PathNotFoundException e) {
                     logger.error("FlowFile {} could not find path {} for attribute key {}.", new Object[]{flowFile.getId(), jsonPathExp.getPath(), jsonPathAttrKey}, e);
                     jsonPathResults.put(jsonPathAttrKey, "");
                 }
+
+                switch (destination) {
+                    case DESTINATION_ATTRIBUTE:
+                        jsonPathResults.put(jsonPathAttrKey, resultHolder.get());
+                        break;
+                    case DESTINATION_CONTENT:
+                        flowFile = processSession.write(flowFile, new OutputStreamCallback() {
+                            @Override
+                            public void process(final OutputStream out) throws IOException {
+                                try (OutputStream outputStream = new BufferedOutputStream(out)) {
+                                    outputStream.write(resultHolder.get().getBytes(StandardCharsets.UTF_8));
+                                }
+                            }
+                        });
+                        break;
+                }
             }
             flowFile = processSession.putAllAttributes(flowFile, jsonPathResults);
             processSession.transfer(flowFile, REL_MATCH);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d4a94c37/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index 60e19d9..6a1fbad 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -118,6 +118,28 @@ public class TestEvaluateJsonPath {
     }
 
     @Test
+    public void testExtractPath_destinationAttributes_twoPaths_notFound() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+
+        String jsonPathIdAttrKey = "evaluatejson.id";
+        String jsonPathNameAttrKey = "evaluatejson.name";
+
+        testRunner.setProperty(jsonPathIdAttrKey, "$[0]._id.nonexistent");
+        testRunner.setProperty(jsonPathNameAttrKey, "$[0].name.nonexistent");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        Relationship expectedRel = EvaluateJsonPath.REL_MATCH;
+
+        testRunner.assertAllFlowFilesTransferred(expectedRel, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0);
+        Assert.assertEquals("Transferred flow file did not have the correct result for id attribute", "", out.getAttribute(jsonPathIdAttrKey));
+        Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", "", out.getAttribute(jsonPathNameAttrKey));
+    }
+
+    @Test
     public void testExtractPath_destinationContent() throws Exception {
         String jsonPathAttrKey = "JsonPath";
 


[21/29] incubator-nifi git commit: Removing unnessecary local variable

Posted by jo...@apache.org.
Removing unnessecary local variable


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1a746212
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1a746212
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1a746212

Branch: refs/heads/NIFI-360
Commit: 1a746212bf2e13680d71dd12dfb60df3a20784ab
Parents: 7a74dd9
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 16:04:21 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 16:04:21 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/processors/standard/util/JsonUtils.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1a746212/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
index 5fe2a56..efe89ab 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -104,13 +104,9 @@ public class JsonUtils {
      * @throws IOException
      */
     public static boolean isValidJson(InputStream inputStream) throws IOException {
-        boolean isValid = false;
-
         try (InputStreamReader inputStreamReader = new InputStreamReader(inputStream)) {
-            isValid = JSONValue.isValidJsonStrict(inputStreamReader);
+            return JSONValue.isValidJsonStrict(inputStreamReader);
         }
-
-        return isValid;
     }
 
     /**


[25/29] incubator-nifi git commit: Adding missing apostrophe for description of SplitJson

Posted by jo...@apache.org.
Adding missing apostrophe for description of SplitJson


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5f647aa4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5f647aa4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5f647aa4

Branch: refs/heads/NIFI-360
Commit: 5f647aa46523ee98c8f8c7339216f852e422f9c7
Parents: 408f636
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 18:40:12 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 18:40:12 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/processors/standard/SplitJson.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5f647aa4/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
index 3ca0ace..c2b520c 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
@@ -41,7 +41,7 @@ import java.util.*;
 @SupportsBatching
 @Tags({"json", "split", "jsonpath"})
 @CapabilityDescription("Splits a JSON File into multiple, separate FlowFiles for an array element specified by a JsonPath expression. "
-        + "Each generated FlowFile is comprised of an element of the specified array and transferred to relationship 'split, "
+        + "Each generated FlowFile is comprised of an element of the specified array and transferred to relationship 'split,' "
         + "with the original file transferred to the 'original' relationship. If the specified JsonPath is not found or "
         + "does not evaluate to an array element, the original file is routed to 'failure' and no files are generated.")
 public class SplitJson extends AbstractProcessor {


[28/29] incubator-nifi git commit: Merge branch 'json-processors' of https://github.com/apiri/incubator-nifi into NIFI-360

Posted by jo...@apache.org.
Merge branch 'json-processors' of https://github.com/apiri/incubator-nifi into NIFI-360


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f05cc938
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f05cc938
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f05cc938

Branch: refs/heads/NIFI-360
Commit: f05cc9383cc86f3697aed59edefff00d35336838
Parents: dde5fd5 9a5b6d5
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 17 22:17:05 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 17 22:17:05 2015 -0500

----------------------------------------------------------------------
 nifi/LICENSE                                    |  28 ++
 .../nifi-standard-processors/pom.xml            |   4 +
 .../processors/standard/EvaluateJsonPath.java   | 240 +++++++++++
 .../nifi/processors/standard/SplitJson.java     | 143 +++++++
 .../processors/standard/util/JsonUtils.java     | 127 ++++++
 .../org.apache.nifi.processor.Processor         |   2 +
 .../standard/TestEvaluateJsonPath.java          | 252 +++++++++++
 .../nifi/processors/standard/TestSplitJson.java | 126 ++++++
 .../test/resources/TestJson/json-sample.json    | 415 +++++++++++++++++++
 nifi/pom.xml                                    |   5 +
 10 files changed, 1342 insertions(+)
----------------------------------------------------------------------



[26/29] incubator-nifi git commit: Removing extraneous comment block that was replaced by method Javadoc

Posted by jo...@apache.org.
Removing extraneous comment block that was replaced by method Javadoc


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c88b427e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c88b427e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c88b427e

Branch: refs/heads/NIFI-360
Commit: c88b427e8201a1e1ba42720def252e0762e31ed9
Parents: 5f647aa
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 18:42:05 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 18:42:05 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/processors/standard/util/JsonUtils.java     | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c88b427e/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
index efe89ab..6f97207 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -89,11 +89,6 @@ public class JsonUtils {
         return contextHolder.get();
     }
 
-    /*
-     * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
-     * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
-     */
-
     /**
      * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition, accordingly a strict JSON approach
      * is preferred in determining whether or not a document is valid.


[07/29] incubator-nifi git commit: Providing validation of the input FlowFile as JSON

Posted by jo...@apache.org.
Providing validation of the input FlowFile as JSON


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c3c4d369
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c3c4d369
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c3c4d369

Branch: refs/heads/NIFI-360
Commit: c3c4d36944fa8a773d042edefa8cfa0f58edcd07
Parents: 40da65f
Author: Aldrin Piri <al...@gmail.com>
Authored: Mon Feb 16 10:56:06 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Mon Feb 16 10:56:06 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJsonPath.java   | 55 +++++++++++++++++---
 .../standard/TestEvaluateJsonPath.java          | 15 +++++-
 2 files changed, 62 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c3c4d369/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 4e1c6ba..41da277 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -16,9 +16,9 @@
  */
 package org.apache.nifi.processors.standard;
 
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.InvalidPathException;
-import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.*;
+import com.jayway.jsonpath.spi.json.JsonProvider;
+import net.minidev.json.JSONValue;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -32,10 +32,11 @@ import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.util.ObjectHolder;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
+import java.io.InputStreamReader;
 import java.util.*;
 
 @EventDriven
@@ -63,6 +64,8 @@ public class EvaluateJsonPath extends AbstractProcessor {
     private Set<Relationship> relationships;
     private List<PropertyDescriptor> properties;
 
+    private static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider();
+
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
@@ -122,19 +125,56 @@ public class EvaluateJsonPath extends AbstractProcessor {
     }
 
     @Override
-    public void onTrigger(ProcessContext processContext, ProcessSession processSession) throws ProcessException {
+    public void onTrigger(ProcessContext processContext, final ProcessSession processSession) throws ProcessException {
 
         final FlowFile flowFile = processSession.get();
         if (flowFile == null) {
             return;
         }
+
+        // Determine the destination
+
+        final String destination = processContext.getProperty(DESTINATION).getValue();
+
+        final ObjectHolder<DocumentContext> contextHolder = new ObjectHolder<>(null);
+
+        // Parse the document once to support multiple path evaluations if specified
         processSession.read(flowFile, new InputStreamCallback() {
             @Override
             public void process(InputStream in) throws IOException {
-                // Parse the document once to support multiple path evaluations if specified
-                Object document = Configuration.defaultConfiguration().jsonProvider().parse(in, StandardCharsets.UTF_8.displayName());
+                /*
+                 * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
+                 * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
+                 */
+                boolean validJson = JSONValue.isValidJsonStrict(new InputStreamReader(in));
+                if (validJson) {
+                    DocumentContext ctx = JsonPath.parse(in);
+                    contextHolder.set(ctx);
+                } else {
+                    getLogger().error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile.getId()});
+                    processSession.transfer(flowFile, REL_FAILURE);
+                }
             }
         });
+
+        DocumentContext documentContext = contextHolder.get();
+
+        if (documentContext == null) {
+            return;
+        }
+
+        try {
+            switch (destination) {
+                case DESTINATION_ATTRIBUTE:
+                    break;
+                case DESTINATION_CONTENT:
+                    break;
+            }
+            processSession.transfer(flowFile, REL_MATCH);
+        } catch (PathNotFoundException e) {
+            getLogger().warn("FlowFile {} could not be read from.", new Object[]{flowFile.getId()}, e);
+            processSession.transfer(flowFile, REL_NO_MATCH);
+        }
     }
 
     private static class JsonPathValidator implements Validator {
@@ -149,4 +189,5 @@ public class EvaluateJsonPath extends AbstractProcessor {
             return new ValidationResult.Builder().valid(error == null).explanation(error).build();
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c3c4d369/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
index 9fb1130..a4e65cf 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.standard;
 
+import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Assert;
@@ -27,6 +28,7 @@ import java.nio.file.Paths;
 public class TestEvaluateJsonPath {
 
     private static final Path JSON_SNIPPET = Paths.get("src/test/resources/TestJson/json-sample.json");
+    private static final Path XML_SNIPPET = Paths.get("src/test/resources/TestXml/xml-snippet.xml");
 
     @Test(expected = AssertionError.class)
     public void testInvalidJsonPath() {
@@ -36,5 +38,16 @@ public class TestEvaluateJsonPath {
 
         Assert.fail("An improper JsonPath expression was not detected as being invalid.");
     }
-    
+
+    @Test
+    public void testInvalidJsonDocument() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+
+        testRunner.enqueue(XML_SNIPPET);
+        testRunner.run();
+
+        testRunner.assertAllFlowFilesTransferred(EvaluateJsonPath.REL_FAILURE, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateJsonPath.REL_FAILURE).get(0);
+    }
 }


[20/29] incubator-nifi git commit: Adding a test case for SplitJson where the specified path cannot be found. Adjusting documentation for additional detail.

Posted by jo...@apache.org.
Adding a test case for SplitJson where the specified path cannot be found.  Adjusting documentation for additional detail.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/7a74dd94
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/7a74dd94
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/7a74dd94

Branch: refs/heads/NIFI-360
Commit: 7a74dd942008bf2178634746e70266f46657d4ef
Parents: 2e05dcb
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 15:50:42 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 15:50:42 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/SplitJson.java     | 12 +++++++--
 .../processors/standard/util/JsonUtils.java     | 28 +++++++++++++++-----
 .../nifi/processors/standard/TestSplitJson.java | 11 ++++++++
 3 files changed, 43 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7a74dd94/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
index e589b48..3ca0ace 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard;
 
 import com.jayway.jsonpath.DocumentContext;
 import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.PathNotFoundException;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -105,10 +106,17 @@ public class SplitJson extends AbstractProcessor {
 
         final List<FlowFile> segments = new ArrayList<>();
 
-        Object jsonPathResult = documentContext.read(jsonPath);
+        Object jsonPathResult;
+        try {
+            jsonPathResult = documentContext.read(jsonPath);
+        } catch (PathNotFoundException e) {
+            logger.warn("JsonPath {} could not be found for FlowFile {}", new Object[]{jsonPath.getPath(), original});
+            processSession.transfer(original, REL_FAILURE);
+            return;
+        }
 
         if (!(jsonPathResult instanceof List)) {
-            logger.error("The evaluated value {} of {} was not an array compatible type and cannot be split.",
+            logger.error("The evaluated value {} of {} was not a JSON Array compatible type and cannot be split.",
                     new Object[]{jsonPathResult, jsonPath.getPath()});
             processSession.transfer(original, REL_FAILURE);
             return;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7a74dd94/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
index 0bf33dd..5fe2a56 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -89,14 +89,23 @@ public class JsonUtils {
         return contextHolder.get();
     }
 
-
+    /*
+     * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
+     * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
+     */
+
+    /**
+     * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition, accordingly a strict JSON approach
+     * is preferred in determining whether or not a document is valid.
+     * Performs a validation of the provided stream according to RFC 4627 as implemented by {@link net.minidev.json.parser.JSONParser#MODE_RFC4627}
+     *
+     * @param inputStream of content to be validated as JSON
+     * @return true, if the content is valid within the bounds of the strictness specified; false otherwise
+     * @throws IOException
+     */
     public static boolean isValidJson(InputStream inputStream) throws IOException {
         boolean isValid = false;
 
-        /*
-         * JSONValue#isValidJson is permissive to the degree of the Smart JSON definition.
-         * Accordingly, a strict JSON approach is preferred in determining whether or not a document is valid.
-         */
         try (InputStreamReader inputStreamReader = new InputStreamReader(inputStream)) {
             isValid = JSONValue.isValidJsonStrict(inputStreamReader);
         }
@@ -104,8 +113,15 @@ public class JsonUtils {
         return isValid;
     }
 
+    /**
+     * Determines the context by which JsonSmartJsonProvider would treat the value.  {@link java.util.Map} and
+     * {@link java.util.List} objects can be rendered as JSON elements, everything else is treated as a scalar.
+     *
+     * @param obj item to be inspected if it is a scalar or a JSON element
+     * @return false, if the object is a supported type; true otherwise
+     */
     public static boolean isJsonScalar(Object obj) {
-        // For the default provider, a Map or List is able to be handled as a JSON entity
+        // For the default provider, JsonSmartJsonProvider, a Map or List is able to be handled as a JSON entity
         return !(obj instanceof Map || obj instanceof List);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7a74dd94/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
index dd6fc6d..f47467f 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java
@@ -112,4 +112,15 @@ public class TestSplitJson {
         testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0).assertContentEquals("{\"first\":\"Shaffer\",\"last\":\"Pearson\"}");
     }
 
+    @Test
+    public void testSplit_pathNotFound() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new SplitJson());
+        testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$.nonexistent");
+
+        testRunner.enqueue(JSON_SNIPPET);
+        testRunner.run();
+
+        testRunner.assertTransferCount(SplitJson.REL_FAILURE, 1);
+        testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).get(0).assertContentEquals(JSON_SNIPPET);
+    }
 }


[05/29] incubator-nifi git commit: Adding an implementation of validation for JsonPath, providing a sample JSON file, and creating an associated test class.

Posted by jo...@apache.org.
Adding an implementation of validation for JsonPath, providing a sample JSON file, and creating an associated test class.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/da6b55f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/da6b55f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/da6b55f3

Branch: refs/heads/NIFI-360
Commit: da6b55f34ced99e3364aa382732a71d2d24b33f5
Parents: e75213e
Author: Aldrin Piri <al...@gmail.com>
Authored: Sat Feb 14 14:13:49 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Sat Feb 14 14:13:49 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/EvaluateJSONPath.java   |  24 +-
 .../standard/TestEvaluateJsonPath.java          |  40 ++
 .../test/resources/TestJson/json-sample.json    | 415 +++++++++++++++++++
 3 files changed, 476 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da6b55f3/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
index 349e623..4e1c6ba 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java
@@ -16,6 +16,9 @@
  */
 package org.apache.nifi.processors.standard;
 
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.InvalidPathException;
+import com.jayway.jsonpath.JsonPath;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -28,7 +31,11 @@ import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.*;
 
 @EventDriven
@@ -121,14 +128,25 @@ public class EvaluateJsonPath extends AbstractProcessor {
         if (flowFile == null) {
             return;
         }
-
+        processSession.read(flowFile, new InputStreamCallback() {
+            @Override
+            public void process(InputStream in) throws IOException {
+                // Parse the document once to support multiple path evaluations if specified
+                Object document = Configuration.defaultConfiguration().jsonProvider().parse(in, StandardCharsets.UTF_8.displayName());
+            }
+        });
     }
 
     private static class JsonPathValidator implements Validator {
-
         @Override
         public ValidationResult validate(String subject, String input, ValidationContext context) {
-            return null;
+            String error = null;
+            try {
+                JsonPath compile = JsonPath.compile(input);
+            } catch (InvalidPathException ipe) {
+                error = ipe.toString();
+            }
+            return new ValidationResult.Builder().valid(error == null).explanation(error).build();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da6b55f3/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
new file mode 100644
index 0000000..9fb1130
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java
@@ -0,0 +1,40 @@
+/*
+ * 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.standard;
+
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class TestEvaluateJsonPath {
+
+    private static final Path JSON_SNIPPET = Paths.get("src/test/resources/TestJson/json-sample.json");
+
+    @Test(expected = AssertionError.class)
+    public void testInvalidJsonPath() {
+        final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath());
+        testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_ATTRIBUTE);
+        testRunner.setProperty("invalid.jsonPath", "$..");
+
+        Assert.fail("An improper JsonPath expression was not detected as being invalid.");
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da6b55f3/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestJson/json-sample.json
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestJson/json-sample.json b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestJson/json-sample.json
new file mode 100644
index 0000000..09de806
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestJson/json-sample.json
@@ -0,0 +1,415 @@
+[
+  {
+    "_id": "54df94072d5dbf7dc6340cc5",
+    "index": 0,
+    "guid": "b9f636cb-b939-42a9-b067-70d286116271",
+    "isActive": true,
+    "balance": "$3,200.07",
+    "picture": "http://placehold.it/32x32",
+    "age": 20,
+    "eyeColor": "brown",
+    "name": {
+      "first": "Shaffer",
+      "last": "Pearson"
+    },
+    "company": "DATAGEN",
+    "email": "shaffer.pearson@datagen.co.uk",
+    "phone": "+1 (972) 588-2272",
+    "address": "662 Rewe Street, Starks, California, 9066",
+    "about": "Aliquip exercitation ad duis irure consectetur magna aliquip amet. Exercitation labore ex laboris non dolor eu. In magna amet non nulla sit laboris do aliqua aliquip. Est elit ipsum ad ea in Lorem mollit Lorem laborum. Ad labore minim aliqua dolore reprehenderit commodo nulla fugiat eiusmod nostrud cillum est. Deserunt minim in non aliqua non.\r\n",
+    "registered": "Wednesday, January 7, 2015 5:51 PM",
+    "latitude": -50.359159,
+    "longitude": -94.01781,
+    "tags": [
+      "ea",
+      "enim",
+      "commodo",
+      "magna",
+      "sunt",
+      "dolore",
+      "aute"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Holloway Kim"
+      },
+      {
+        "id": 1,
+        "name": "Clark Medina"
+      },
+      {
+        "id": 2,
+        "name": "Rosemarie Salazar"
+      }
+    ],
+    "greeting": "Hello, Shaffer! You have 9 unread messages.",
+    "favoriteFruit": "apple"
+  },
+  {
+    "_id": "54df94073ab1785758096418",
+    "index": 1,
+    "guid": "fda79e72-6489-41f5-bbd5-a5e7d2996dda",
+    "isActive": false,
+    "balance": "$1,416.15",
+    "picture": "http://placehold.it/32x32",
+    "age": 38,
+    "eyeColor": "blue",
+    "name": {
+      "first": "Frazier",
+      "last": "Ramsey"
+    },
+    "company": "STREZZO",
+    "email": "frazier.ramsey@strezzo.biz",
+    "phone": "+1 (909) 448-2724",
+    "address": "624 Cedar Street, Iola, North Carolina, 2827",
+    "about": "Sit sunt eiusmod irure ipsum Lorem irure aliquip cupidatat in proident dolore sunt adipisicing. Aute ipsum reprehenderit aute aliquip ad id pariatur dolor dolore et exercitation. Pariatur est adipisicing eu aliqua ea sint qui. Fugiat officia voluptate anim dolore cupidatat amet. Amet cillum dolor magna elit fugiat.\r\n",
+    "registered": "Sunday, January 5, 2014 1:18 PM",
+    "latitude": -14.729254,
+    "longitude": 126.396861,
+    "tags": [
+      "non",
+      "laboris",
+      "nulla",
+      "commodo",
+      "nostrud",
+      "qui",
+      "ea"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Valenzuela Stone"
+      },
+      {
+        "id": 1,
+        "name": "King Munoz"
+      },
+      {
+        "id": 2,
+        "name": "Kari Woodard"
+      }
+    ],
+    "greeting": "Hello, Frazier! You have 7 unread messages.",
+    "favoriteFruit": "strawberry"
+  },
+  {
+    "_id": "54df9407369a4d3f1b4aed39",
+    "index": 2,
+    "guid": "b6a68edb-4ddd-487b-b104-f02bec805e4c",
+    "isActive": true,
+    "balance": "$2,487.31",
+    "picture": "http://placehold.it/32x32",
+    "age": 27,
+    "eyeColor": "green",
+    "name": {
+      "first": "Cindy",
+      "last": "Shepherd"
+    },
+    "company": "EMTRAK",
+    "email": "cindy.shepherd@emtrak.org",
+    "phone": "+1 (867) 466-3223",
+    "address": "659 Colin Place, Vaughn, Washington, 1106",
+    "about": "Nulla sunt aliquip eiusmod occaecat duis officia eiusmod aliqua cillum ut. Irure eu est nulla dolor laborum eiusmod Lorem dolore culpa aliquip veniam duis. Sint cupidatat laboris commodo sunt consequat ullamco culpa ad labore. Velit do voluptate quis occaecat ex ipsum cupidatat occaecat dolor officia laborum labore.\r\n",
+    "registered": "Thursday, June 26, 2014 9:56 PM",
+    "latitude": 85.829527,
+    "longitude": -79.452723,
+    "tags": [
+      "cillum",
+      "do",
+      "veniam",
+      "dolore",
+      "voluptate",
+      "et",
+      "adipisicing"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Decker Carver"
+      },
+      {
+        "id": 1,
+        "name": "Donaldson Burgess"
+      },
+      {
+        "id": 2,
+        "name": "Santana Heath"
+      }
+    ],
+    "greeting": "Hello, Cindy! You have 8 unread messages.",
+    "favoriteFruit": "strawberry"
+  },
+  {
+    "_id": "54df94076f342042d027ca67",
+    "index": 3,
+    "guid": "ac591519-1642-4092-9646-17b4b7a9e38b",
+    "isActive": false,
+    "balance": "$3,480.12",
+    "picture": "http://placehold.it/32x32",
+    "age": 37,
+    "eyeColor": "green",
+    "name": {
+      "first": "Colon",
+      "last": "Gamble"
+    },
+    "company": "RONELON",
+    "email": "colon.gamble@ronelon.net",
+    "phone": "+1 (988) 431-2933",
+    "address": "472 Ryerson Street, Gwynn, Wyoming, 4200",
+    "about": "Ad duis nostrud laboris id aute reprehenderit veniam aute aute laborum exercitation laborum. In minim quis in sunt minim labore deserunt id dolor ea sit. Ipsum tempor Lorem aliqua ad sit quis duis exercitation quis. Dolore voluptate aute ut est non quis do aute exercitation consectetur reprehenderit proident quis.\r\n",
+    "registered": "Tuesday, July 29, 2014 1:38 PM",
+    "latitude": -9.922105,
+    "longitude": -170.581901,
+    "tags": [
+      "fugiat",
+      "incididunt",
+      "proident",
+      "laboris",
+      "id",
+      "ullamco",
+      "non"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Shawn Collins"
+      },
+      {
+        "id": 1,
+        "name": "Holland West"
+      },
+      {
+        "id": 2,
+        "name": "Daniel Fischer"
+      }
+    ],
+    "greeting": "Hello, Colon! You have 7 unread messages.",
+    "favoriteFruit": "strawberry"
+  },
+  {
+    "_id": "54df94075774d288fc86a912",
+    "index": 4,
+    "guid": "daec0340-7900-4a65-92fc-22e727577660",
+    "isActive": true,
+    "balance": "$3,042.74",
+    "picture": "http://placehold.it/32x32",
+    "age": 36,
+    "eyeColor": "brown",
+    "name": {
+      "first": "Carter",
+      "last": "Russo"
+    },
+    "company": "NORALEX",
+    "email": "carter.russo@noralex.biz",
+    "phone": "+1 (819) 543-3605",
+    "address": "147 Everit Street, Saticoy, Missouri, 5963",
+    "about": "Ea irure non pariatur ipsum. Magna eu enim anim Lorem quis sint cillum. Voluptate proident commodo dolor aute consectetur reprehenderit dolor nostrud ipsum cillum magna dolor. Reprehenderit sit consequat pariatur enim do occaecat exercitation reprehenderit.\r\n",
+    "registered": "Saturday, January 25, 2014 10:12 PM",
+    "latitude": -65.101248,
+    "longitude": 19.867506,
+    "tags": [
+      "dolore",
+      "et",
+      "ex",
+      "eu",
+      "nostrud",
+      "ex",
+      "ad"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Felicia Hull"
+      },
+      {
+        "id": 1,
+        "name": "Jerri Mays"
+      },
+      {
+        "id": 2,
+        "name": "Jo Justice"
+      }
+    ],
+    "greeting": "Hello, Carter! You have 7 unread messages.",
+    "favoriteFruit": "apple"
+  },
+  {
+    "_id": "54df940741be468e58e87dd3",
+    "index": 5,
+    "guid": "16a037a3-fe30-4c51-8d09-f24ad54f4719",
+    "isActive": true,
+    "balance": "$1,979.92",
+    "picture": "http://placehold.it/32x32",
+    "age": 20,
+    "eyeColor": "blue",
+    "name": {
+      "first": "Claudia",
+      "last": "Houston"
+    },
+    "company": "FISHLAND",
+    "email": "claudia.houston@fishland.com",
+    "phone": "+1 (860) 498-3802",
+    "address": "821 Remsen Avenue, Ada, Vermont, 3101",
+    "about": "Lorem eu deserunt et non id consectetur laborum voluptate id magna labore. Dolore enim voluptate mollit culpa cupidatat officia do aute voluptate Lorem commodo. Nisi nostrud amet in labore ullamco nisi magna adipisicing voluptate aliquip qui consequat enim. Pariatur adipisicing nostrud ut deserunt ad excepteur. Lorem do voluptate adipisicing et laborum commodo nulla excepteur laborum quis tempor proident velit.\r\n",
+    "registered": "Thursday, August 7, 2014 7:48 AM",
+    "latitude": 34.6075,
+    "longitude": -2.643176,
+    "tags": [
+      "enim",
+      "eu",
+      "sint",
+      "qui",
+      "elit",
+      "laboris",
+      "commodo"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Boyd Morrison"
+      },
+      {
+        "id": 1,
+        "name": "Wendi Sandoval"
+      },
+      {
+        "id": 2,
+        "name": "Mindy Bush"
+      }
+    ],
+    "greeting": "Hello, Claudia! You have 8 unread messages.",
+    "favoriteFruit": "apple"
+  },
+  {
+    "_id": "54df9407fbfc2103751de2e7",
+    "index": 6,
+    "guid": "60241980-5362-41dd-b6e5-e55f174904cf",
+    "isActive": true,
+    "balance": "$3,106.83",
+    "picture": "http://placehold.it/32x32",
+    "age": 40,
+    "eyeColor": "green",
+    "name": {
+      "first": "Beulah",
+      "last": "Myers"
+    },
+    "company": "UNI",
+    "email": "beulah.myers@uni.tv",
+    "phone": "+1 (969) 407-3571",
+    "address": "661 Matthews Court, Osage, Delaware, 1167",
+    "about": "Officia ipsum reprehenderit in nostrud Lorem labore consectetur nulla quis officia ullamco. Eiusmod ipsum deserunt consectetur cillum et duis do esse veniam occaecat Lorem dolor consequat. Lorem esse cupidatat aute et ut.\r\n",
+    "registered": "Sunday, January 25, 2015 8:22 PM",
+    "latitude": 72.620891,
+    "longitude": 155.859974,
+    "tags": [
+      "minim",
+      "fugiat",
+      "irure",
+      "culpa",
+      "exercitation",
+      "labore",
+      "commodo"
+    ],
+    "range": [
+      0,
+      1,
+      2,
+      3,
+      4,
+      5,
+      6,
+      7,
+      8,
+      9
+    ],
+    "friends": [
+      {
+        "id": 0,
+        "name": "Corina Francis"
+      },
+      {
+        "id": 1,
+        "name": "Vera Carson"
+      },
+      {
+        "id": 2,
+        "name": "Blevins Camacho"
+      }
+    ],
+    "greeting": "Hello, Beulah! You have 8 unread messages.",
+    "favoriteFruit": "apple"
+  }
+]
\ No newline at end of file


[27/29] incubator-nifi git commit: Encapsulating the rendering of a JsonPath result within JsonUtils

Posted by jo...@apache.org.
Encapsulating the rendering of a JsonPath result within JsonUtils


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/9a5b6d5b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/9a5b6d5b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/9a5b6d5b

Branch: refs/heads/NIFI-360
Commit: 9a5b6d5ba234c471df8faefb0ebf9470a68bebb5
Parents: c88b427
Author: Aldrin Piri <al...@gmail.com>
Authored: Tue Feb 17 21:29:40 2015 -0500
Committer: Aldrin Piri <al...@gmail.com>
Committed: Tue Feb 17 21:29:40 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/processors/standard/EvaluateJsonPath.java | 10 +---------
 .../org/apache/nifi/processors/standard/SplitJson.java    |  7 +------
 .../apache/nifi/processors/standard/util/JsonUtils.java   | 10 +++++++++-
 3 files changed, 11 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9a5b6d5b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 3a64a34..d79a6de 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -215,7 +215,7 @@ public class EvaluateJsonPath extends AbstractProcessor {
                     }
                 }
 
-                final String resultRepresentation = getResultRepresentation(resultHolder.get());
+                final String resultRepresentation = JsonUtils.getResultRepresentation(resultHolder.get());
                 switch (destination) {
                     case DESTINATION_ATTRIBUTE:
                         jsonPathResults.put(jsonPathAttrKey, resultRepresentation);
@@ -237,12 +237,4 @@ public class EvaluateJsonPath extends AbstractProcessor {
         }
     }
 
-    private static String getResultRepresentation(Object jsonPathResult) {
-        if (JsonUtils.isJsonScalar(jsonPathResult)) {
-            return jsonPathResult.toString();
-        }
-        return JsonUtils.JSON_PROVIDER.toJson(jsonPathResult);
-    }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9a5b6d5b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
index c2b520c..78e1b2a 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
@@ -129,12 +129,7 @@ public class SplitJson extends AbstractProcessor {
             split = processSession.write(split, new OutputStreamCallback() {
                 @Override
                 public void process(OutputStream out) throws IOException {
-                    String resultSegmentContent;
-                    if (JsonUtils.isJsonScalar(resultSegment)) {
-                        resultSegmentContent = resultSegment.toString();
-                    } else {
-                        resultSegmentContent = JsonUtils.JSON_PROVIDER.toJson(resultSegment);
-                    }
+                    String resultSegmentContent = JsonUtils.getResultRepresentation(resultSegment);
                     out.write(resultSegmentContent.getBytes(StandardCharsets.UTF_8));
                 }
             });

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9a5b6d5b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
index 6f97207..2174c1e 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java
@@ -46,7 +46,7 @@ import java.util.Map;
  */
 public class JsonUtils {
 
-    public static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider();
+    static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider();
 
     public static final Validator JSON_PATH_VALIDATOR = new Validator() {
         @Override
@@ -116,4 +116,12 @@ public class JsonUtils {
         return !(obj instanceof Map || obj instanceof List);
     }
 
+
+    public static String getResultRepresentation(Object jsonPathResult) {
+        if (JsonUtils.isJsonScalar(jsonPathResult)) {
+            return jsonPathResult.toString();
+        }
+        return JSON_PROVIDER.toJson(jsonPathResult);
+    }
+
 }