From 446fc2ef8099faff786f11c7d640232191dac2b0 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Fri, 13 Feb 2015 23:42:40 -0500 Subject: [PATCH 01/43] Adding JSONPath as a managed dependency and including it in the standard processor pom --- .../nifi-standard-bundle/nifi-standard-processors/pom.xml | 4 ++++ nifi/pom.xml | 5 +++++ 2 files changed, 9 insertions(+) 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 a482e19508..5a15cf4053 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 @@ -150,6 +150,10 @@ org.apache.activemq activemq-client + + com.jayway.jsonpath + json-path + org.apache.nifi nifi-ssl-context-service diff --git a/nifi/pom.xml b/nifi/pom.xml index 8539ae9f25..20f3a42dfe 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -783,6 +783,11 @@ nifi-write-ahead-log 0.0.2-incubating-SNAPSHOT + + com.jayway.jsonpath + json-path + 1.2.0 + From 5f03fb11d93791ea9cdde43ffb97bceaf22c0143 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 14 Feb 2015 12:56:19 -0500 Subject: [PATCH 02/43] Creating stub for an EvaluateJSONPath processor and providing configuration akin to EvaluateXPath --- .../processors/standard/EvaluateJSONPath.java | 59 +++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100644 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 0000000000..9ad3e491d6 --- /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 { + + } + +} From ec669e5b42e58335ab511dfa8dfd52bd0848f088 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 14 Feb 2015 13:26:08 -0500 Subject: [PATCH 03/43] Stubbing out validator, setting up relationships and properties. --- .../processors/standard/EvaluateJSONPath.java | 95 +++++++++++++++++-- 1 file changed, 85 insertions(+), 10 deletions(-) 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 9ad3e491d6..349e623308 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 relationships; + private List properties; + + + @Override + protected void init(final ProcessorInitializationContext context) { + final Set relationships = new HashSet<>(); + relationships.add(REL_MATCH); + relationships.add(REL_NO_MATCH); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + + final List properties = new ArrayList<>(); + properties.add(DESTINATION); + this.properties = Collections.unmodifiableList(properties); + } + + @Override + protected Collection customValidate(final ValidationContext context) { + final List 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 getRelationships() { + return relationships; + } + + @Override + protected List 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; + } + } } From e75213eead936baa7046d9c9ba5b86a4508f532a Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 14 Feb 2015 13:35:53 -0500 Subject: [PATCH 04/43] Adding EvaluateJsonPath to the Processor services file --- .../META-INF/services/org.apache.nifi.processor.Processor | 1 + 1 file changed, 1 insertion(+) 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 66c96970e6..8a1fd740f1 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 From da6b55f34ced99e3364aa382732a71d2d24b33f5 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 14 Feb 2015 14:13:49 -0500 Subject: [PATCH 05/43] Adding an implementation of validation for JsonPath, providing a sample JSON file, and creating an associated test class. --- .../processors/standard/EvaluateJSONPath.java | 24 +- .../standard/TestEvaluateJsonPath.java | 40 ++ .../test/resources/TestJson/json-sample.json | 415 ++++++++++++++++++ 3 files changed, 476 insertions(+), 3 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateJsonPath.java create mode 100644 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/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 349e623308..4e1c6ba45e 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(); } } } 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 0000000000..9fb1130645 --- /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."); + } + +} 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 0000000000..09de8062e6 --- /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 From 40da65f1937089ea3935f21da384f4bbbc36d20d Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 15 Feb 2015 15:25:22 -0500 Subject: [PATCH 06/43] Adjusting filename of EvaluateJsonPath --- .../standard/{EvaluateJSONPath.java => EvaluateJsonPath.java} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/{EvaluateJSONPath.java => EvaluateJsonPath.java} (100%) 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 similarity index 100% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJSONPath.java rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java From c3c4d36944fa8a773d042edefa8cfa0f58edcd07 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 10:56:06 -0500 Subject: [PATCH 07/43] Providing validation of the input FlowFile as JSON --- .../processors/standard/EvaluateJsonPath.java | 55 ++++++++++++++++--- .../standard/TestEvaluateJsonPath.java | 15 ++++- 2 files changed, 62 insertions(+), 8 deletions(-) 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 4e1c6ba45e..41da277b94 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 relationships; private List 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 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(); } } + } 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 9fb1130645..a4e65cf517 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); + } } From b3328490c65d0b08bed4f711b6990d10262bbef2 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:03:24 -0500 Subject: [PATCH 08/43] Completing initial functionality of EvaluateJsonPath and associated tests. --- .../processors/standard/EvaluateJsonPath.java | 149 +++++++++++++----- .../standard/TestEvaluateJsonPath.java | 44 ++++++ 2 files changed, 156 insertions(+), 37 deletions(-) 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 41da277b94..e74783806a 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 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 flowFiles = processSession.get(50); + if (flowFiles.isEmpty()) { return; } - // Determine the destination + final ProcessorLog logger = getLogger(); + + /* Build the JsonPath expressions from attributes */ + final Map attributeToJsonPathMap = new HashMap<>(); + + for (final Map.Entry 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 contextHolder = new ObjectHolder<>(null); + 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); + } + } + }); - // 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); + if (!validJsonHolder.get()) { + logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile.getId()}); + processSession.transfer(flowFile, REL_FAILURE); + continue flowFileLoop; + } + + // Parse the document once into an associated context to support multiple path evaluations if specified + final ObjectHolder 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 jsonPathResults = new HashMap<>(); + + // Iterate through all JsonPath entries specified + for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { + + String jsonPathAttrKey = attributeJsonPathEntry.getKey(); + JsonPath jsonPathExp = attributeJsonPathEntry.getValue(); + final String evalResult = evaluatePathForContext(jsonPathExp, documentContext); + + 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, ""); } } - }); - - DocumentContext documentContext = contextHolder.get(); - - if (documentContext == null) { - return; - } - - try { - switch (destination) { - case DESTINATION_ATTRIBUTE: - break; - case DESTINATION_CONTENT: - break; - } + 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) { 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 a4e65cf517..7174e9a270 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)); + } } From 974617d44ee41bd5cac9f238eca43cb9f81684a6 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:09:22 -0500 Subject: [PATCH 09/43] Adding a test to verify the placement of a JSON path expression with a content destination. --- .../standard/TestEvaluateJsonPath.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 7174e9a270..308440fcff 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"); + } } From 78ad0a3147cf1f3573f94f79685ebb868b5d190d Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:20:31 -0500 Subject: [PATCH 10/43] Adding a test for multiple attribute paths --- .../standard/TestEvaluateJsonPath.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) 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 308440fcff..60e19d91c4 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 @@ -95,6 +95,28 @@ public class TestEvaluateJsonPath { Assert.assertEquals("Transferred flow file did not have the correct result", "54df94072d5dbf7dc6340cc5", out.getAttribute(jsonPathAttrKey)); } + @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"; From d4a94c37eea9c4a545e9f764c113a3d863257e57 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:24:57 -0500 Subject: [PATCH 11/43] Adding a test for multiple attributes where neither evaluates to a found path. --- .../processors/standard/EvaluateJsonPath.java | 35 ++++++++++--------- .../standard/TestEvaluateJsonPath.java | 22 ++++++++++++ 2 files changed, 41 insertions(+), 16 deletions(-) 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 e74783806a..13eb50b758 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 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); 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 60e19d91c4..6a1fbad460 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 @@ -117,6 +117,28 @@ public class TestEvaluateJsonPath { 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_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"; From 6897090771434fc1553b722b835c3b67e8dd14a9 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:26:35 -0500 Subject: [PATCH 12/43] Adding a test for a destination of attributes and only one JsonPath expression evaluates to a found result. --- .../standard/TestEvaluateJsonPath.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) 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 6a1fbad460..fc125c2e8b 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 @@ -139,6 +139,28 @@ public class TestEvaluateJsonPath { Assert.assertEquals("Transferred flow file did not have the correct result for name attribute", "", out.getAttribute(jsonPathNameAttrKey)); } + @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"; From 7e581307b7d65da79464914b09fe8a90dd5046e9 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:33:48 -0500 Subject: [PATCH 13/43] Adding tests for indefinite results with and without the usage of operators. --- .../standard/TestEvaluateJsonPath.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) 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 fc125c2e8b..bece3da175 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]"); + } } From 5b145e10e8508bec5293aab7318978fc3c9e2b57 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 18:34:23 -0500 Subject: [PATCH 14/43] Adjusting error where the return type of string was omitted from the possible selections --- .../org/apache/nifi/processors/standard/EvaluateJsonPath.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 13eb50b758..d6bcdc184c 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(); From bcebba6632165b6d69eb88ad12895e7678079267 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 22:44:06 -0500 Subject: [PATCH 15/43] Refining logic of how errors are handled on a per destination basis. Adding supporting tests to ensure contract is met. --- .../processors/standard/EvaluateJsonPath.java | 64 +++++++++++++------ .../standard/TestEvaluateJsonPath.java | 20 +++++- 2 files changed, 62 insertions(+), 22 deletions(-) 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 d6bcdc184c..b82764d3ff 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 jsonPathResults = new HashMap<>(); // Iterate through all JsonPath entries specified + jsonPathEvalLoop: for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { String jsonPathAttrKey = attributeJsonPathEntry.getKey(); JsonPath jsonPathExp = attributeJsonPathEntry.getValue(); - final ObjectHolder resultHolder = new ObjectHolder<>(""); + final ObjectHolder 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); - /* - * 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(); + private static String getResultRepresentation(Object jsonPathResult) { + if (isScalar(jsonPathResult)) { + return jsonPathResult.toString(); } - return JSON_PROVIDER.toJson(pathResult); + return JSON_PROVIDER.toJson(jsonPathResult); + } + + private static boolean isScalar(Object obj) { + /* + * A given path could be a JSON object or a single/scalar value + */ + return (obj instanceof String); } private static class JsonPathValidator implements Validator { 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 bece3da175..e72d0690b7 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); + } + } From 0a19ada0a3229b1c8a72fe638f286db33b6f7ba6 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 16 Feb 2015 23:26:45 -0500 Subject: [PATCH 16/43] Adjusting comments and providing a test to ensure a non-scalar value for a specified scalar return type is routed to failure. --- .../processors/standard/EvaluateJsonPath.java | 31 +++++++++---------- .../standard/TestEvaluateJsonPath.java | 19 ++++++++++++ 2 files changed, 34 insertions(+), 16 deletions(-) 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 b82764d3ff..fe89635dca 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 jsonPathResults = new HashMap<>(); - // Iterate through all JsonPath entries specified jsonPathEvalLoop: for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { @@ -226,8 +227,9 @@ public class EvaluateJsonPath extends AbstractProcessor { final ObjectHolder 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); } 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 e72d0690b7..c873969a7d 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); + } + } From 5a81f19b25bc073ec1089c115b6e6fcf01056f91 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 09:44:30 -0500 Subject: [PATCH 17/43] Moving the JsonPath Validator to the JsonPathUtil class so that it can be reused by other processors. --- .../processors/standard/EvaluateJsonPath.java | 21 +++------ .../standard/util/JsonPathUtils.java | 44 +++++++++++++++++++ 2 files changed, 50 insertions(+), 15 deletions(-) create mode 100644 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/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index fe89635dca..05d31dafea 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(); - } - } } 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 0000000000..ecd2ad0358 --- /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 https://github.com/jayway/JsonPath + */ +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(); + } + }; +} From 59ad1948519fa1f8acccfb6eafbda04684edbc96 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 10:26:18 -0500 Subject: [PATCH 18/43] Refactoring common JSON/JsonPath functionality into JsonUtils --- .../processors/standard/EvaluateJsonPath.java | 52 ++------- .../standard/util/JsonPathUtils.java | 44 -------- .../processors/standard/util/JsonUtils.java | 104 ++++++++++++++++++ 3 files changed, 111 insertions(+), 89 deletions(-) delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathUtils.java create mode 100644 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/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index 05d31dafea..0bdd20c502 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 relationships; private List 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 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 jsonPathResults = new HashMap<>(); jsonPathEvalLoop: @@ -226,7 +189,6 @@ public class EvaluateJsonPath extends AbstractProcessor { String jsonPathAttrKey = attributeJsonPathEntry.getKey(); JsonPath jsonPathExp = attributeJsonPathEntry.getValue(); - final ObjectHolder 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) { 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 ecd2ad0358..0000000000 --- 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 https://github.com/jayway/JsonPath - */ -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(); - } - }; -} 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 0000000000..6eb567ec2a --- /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 http://json.org + * @see https://github.com/jayway/JsonPath + */ +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 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; + } +} From 2e05dcbbfdb37843456dff121d9878d8679d7067 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 15:15:53 -0500 Subject: [PATCH 19/43] Providing a SplitJson processor which will break JSON Arrays into their individual elements. Refactored supporting JsonUtils code and EvaluateJsonPath to reuse common functionality. --- .../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 +- .../processors/standard/TestSplitJson.java | 115 ++++++++++++++ 6 files changed, 268 insertions(+), 7 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java create mode 100644 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/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 0bdd20c502..8cdf1a2f33 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 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); - } - } 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 0000000000..e589b487ca --- /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 properties; + private Set relationships; + + @Override + protected void init(final ProcessorInitializationContext context) { + final List properties = new ArrayList<>(); + properties.add(ARRAY_JSON_PATH_EXPRESSION); + this.properties = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_ORIGINAL); + relationships.add(REL_SPLIT); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + protected List 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 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()}); + } +} 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 6eb567ec2a..0bf33dd332 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); + } + } 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 8a1fd740f1..17b5364bd1 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 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 c873969a7d..b7b51033b0 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 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 0000000000..dd6fc6d703 --- /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\"}"); + } + +} From 7a74dd942008bf2178634746e70266f46657d4ef Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 15:50:42 -0500 Subject: [PATCH 20/43] Adding a test case for SplitJson where the specified path cannot be found. Adjusting documentation for additional detail. --- .../nifi/processors/standard/SplitJson.java | 12 +++++++-- .../processors/standard/util/JsonUtils.java | 26 +++++++++++++++---- .../processors/standard/TestSplitJson.java | 11 ++++++++ 3 files changed, 42 insertions(+), 7 deletions(-) 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 e589b487ca..3ca0ace098 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 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; 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 0bf33dd332..5fe2a5664d 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); } 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 dd6fc6d703..f47467f28d 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); + } } From 1a746212bf2e13680d71dd12dfb60df3a20784ab Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 16:04:21 -0500 Subject: [PATCH 21/43] Removing unnessecary local variable --- .../org/apache/nifi/processors/standard/util/JsonUtils.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) 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 5fe2a5664d..efe89aba7a 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; } /** From 627bd91fa719748b850b6c610a89b44be3e6f35e Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 16:15:33 -0500 Subject: [PATCH 22/43] Adding an auto return type for EvaluateJsonPath to match the semantics of the EvaluateXPath processor. --- .../nifi/processors/standard/EvaluateJsonPath.java | 10 +++++++--- .../nifi/processors/standard/TestEvaluateJsonPath.java | 1 + 2 files changed, 8 insertions(+), 3 deletions(-) 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 8cdf1a2f33..093454d892 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) { 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 b7b51033b0..c5ff81476d 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"; From a058fd0460dd5ca91560b1239dd733da37cfa782 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 16:58:04 -0500 Subject: [PATCH 23/43] Adding license terms for asm dependency introduced via JsonPath -> Json-Smart --- nifi/LICENSE | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/nifi/LICENSE b/nifi/LICENSE index 2e67c09d68..4d8be01ccd 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 From 408f636361cbad025752978e10775651b7daddf4 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 17:01:06 -0500 Subject: [PATCH 24/43] Preferring the constant empty string for clarity instead of quotes. --- .../org/apache/nifi/processors/standard/EvaluateJsonPath.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 093454d892..3a64a34a42 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); From 5f647aa46523ee98c8f8c7339216f852e422f9c7 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 18:40:12 -0500 Subject: [PATCH 25/43] Adding missing apostrophe for description of SplitJson --- .../java/org/apache/nifi/processors/standard/SplitJson.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3ca0ace098..c2b520cbe4 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 { From c88b427e8201a1e1ba42720def252e0762e31ed9 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 18:42:05 -0500 Subject: [PATCH 26/43] Removing extraneous comment block that was replaced by method Javadoc --- .../org/apache/nifi/processors/standard/util/JsonUtils.java | 5 ----- 1 file changed, 5 deletions(-) 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 efe89aba7a..6f97207daa 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. From 9a5b6d5ba234c471df8faefb0ebf9470a68bebb5 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Tue, 17 Feb 2015 21:29:40 -0500 Subject: [PATCH 27/43] Encapsulating the rendering of a JsonPath result within JsonUtils --- .../nifi/processors/standard/EvaluateJsonPath.java | 10 +--------- .../org/apache/nifi/processors/standard/SplitJson.java | 7 +------ .../nifi/processors/standard/util/JsonUtils.java | 10 +++++++++- 3 files changed, 11 insertions(+), 16 deletions(-) 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 3a64a34a42..d79a6dec5b 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); - } - - } 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 c2b520cbe4..78e1b2a950 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)); } }); 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 6f97207daa..2174c1ef7e 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); + } + } From 2862771235eb7a9e007c1a14096098d9809987be Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 17 Feb 2015 23:29:16 -0500 Subject: [PATCH 28/43] NIFI-360 providing review feedback on github PR --- 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(-) diff --git a/nifi/LICENSE b/nifi/LICENSE index 4d8be01ccd..50150e3537 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 diff --git a/nifi/nifi-assembly/LICENSE b/nifi/nifi-assembly/LICENSE index da81c41a46..bf9e6490ea 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. diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index 7d42538418..1667e9eaa4 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 ************************ 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 44bb0c08f2..36ba155492 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. 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 c5a15334a7..d63a74ddb7 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 From 81234f3a6dd1effd4c66bcbdbe8d119843b7fdf2 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Fri, 20 Feb 2015 16:05:16 -0500 Subject: [PATCH 29/43] Removing the batched get of flowfiles to utilize the framework provided batching support --- .../processors/standard/EvaluateJsonPath.java | 115 +++++++++--------- 1 file changed, 55 insertions(+), 60 deletions(-) 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 d79a6dec5b..65266fffd3 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 @@ -151,8 +151,8 @@ public class EvaluateJsonPath extends AbstractProcessor { @Override public void onTrigger(ProcessContext processContext, final ProcessSession processSession) throws ProcessException { - List flowFiles = processSession.get(50); - if (flowFiles.isEmpty()) { + FlowFile flowFile = processSession.get(); + if (flowFile == null) { return; } @@ -175,66 +175,61 @@ public class EvaluateJsonPath extends AbstractProcessor { returnType = destination.equals(DESTINATION_CONTENT) ? RETURN_TYPE_JSON : RETURN_TYPE_SCALAR; } - flowFileLoop: - for (FlowFile flowFile : flowFiles) { + final DocumentContext documentContext = JsonUtils.validateAndEstablishJsonContext(processSession, flowFile); - 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; - } - - final Map jsonPathResults = new HashMap<>(); - - jsonPathEvalLoop: - for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { - - String jsonPathAttrKey = attributeJsonPathEntry.getKey(); - JsonPath jsonPathExp = attributeJsonPathEntry.getValue(); - - final ObjectHolder resultHolder = new ObjectHolder<>(null); - try { - Object result = documentContext.read(jsonPathExp); - 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); - continue flowFileLoop; - } - resultHolder.set(result); - } 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, StringUtils.EMPTY); - continue jsonPathEvalLoop; - } else { - processSession.transfer(flowFile, REL_NO_MATCH); - continue flowFileLoop; - } - } - - final String resultRepresentation = JsonUtils.getResultRepresentation(resultHolder.get()); - switch (destination) { - case DESTINATION_ATTRIBUTE: - 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(resultRepresentation.getBytes(StandardCharsets.UTF_8)); - } - } - }); - break; - } - } - flowFile = processSession.putAllAttributes(flowFile, jsonPathResults); - processSession.transfer(flowFile, REL_MATCH); + if (documentContext == null) { + logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile}); + processSession.transfer(flowFile, REL_FAILURE); + return; } + + final Map jsonPathResults = new HashMap<>(); + + jsonPathEvalLoop: + for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { + + String jsonPathAttrKey = attributeJsonPathEntry.getKey(); + JsonPath jsonPathExp = attributeJsonPathEntry.getValue(); + + final ObjectHolder resultHolder = new ObjectHolder<>(null); + try { + Object result = documentContext.read(jsonPathExp); + 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); + return; + } + resultHolder.set(result); + } 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, StringUtils.EMPTY); + continue jsonPathEvalLoop; + } else { + processSession.transfer(flowFile, REL_NO_MATCH); + return; + } + } + + final String resultRepresentation = JsonUtils.getResultRepresentation(resultHolder.get()); + switch (destination) { + case DESTINATION_ATTRIBUTE: + jsonPathResults.put(jsonPathAttrKey, resultRepresentation); + 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(resultRepresentation.getBytes(StandardCharsets.UTF_8)); + } + } + }); + break; + } + } + flowFile = processSession.putAllAttributes(flowFile, jsonPathResults); + processSession.transfer(flowFile, REL_MATCH); } } From 46bf048b2407ddbd61024d2d0e69beccb2f72014 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 21 Feb 2015 20:26:58 -0500 Subject: [PATCH 30/43] Adding an abstract class to serve as a base class for JsonPath related processors and preferring this for much of the functionality present in JsonUtils. --- .../standard/AbstractJsonPathProcessor.java | 110 ++++++++++++++++++ .../processors/standard/EvaluateJsonPath.java | 17 +-- .../nifi/processors/standard/SplitJson.java | 14 ++- .../processors/standard/util/JsonUtils.java | 82 +------------ 4 files changed, 129 insertions(+), 94 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java new file mode 100644 index 0000000000..cb4dcf6ab2 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -0,0 +1,110 @@ +/* + * 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.DocumentContext; +import com.jayway.jsonpath.InvalidPathException; +import com.jayway.jsonpath.JsonPath; +import com.jayway.jsonpath.spi.json.JsonProvider; +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.AbstractProcessor; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processors.standard.util.JsonUtils; +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.util.List; +import java.util.Map; + +/** + * Provides common functionality used for processors interacting and manipulating JSON data via JsonPath. + * + * @see http://json.org + * @see https://github.com/jayway/JsonPath + */ +public abstract class AbstractJsonPathProcessor extends AbstractProcessor { + + protected 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 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(); + } + + /** + * 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 + */ + static boolean isJsonScalar(Object obj) { + // 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); + } + + public static String getResultRepresentation(Object jsonPathResult) { + if (isJsonScalar(jsonPathResult)) { + return jsonPathResult.toString(); + } + return JSON_PROVIDER.toJson(jsonPathResult); + } +} 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 65266fffd3..f7caa68f03 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,10 +29,12 @@ import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.*; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; 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; @@ -57,7 +59,7 @@ import java.util.*; + "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 class EvaluateJsonPath extends AbstractJsonPathProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; public static final String DESTINATION_CONTENT = "flowfile-content"; @@ -142,7 +144,7 @@ public class EvaluateJsonPath extends AbstractProcessor { return new PropertyDescriptor.Builder() .name(propertyDescriptorName) .expressionLanguageSupported(false) - .addValidator(JsonUtils.JSON_PATH_VALIDATOR) + .addValidator(JSON_PATH_VALIDATOR) .required(false) .dynamic(true) .build(); @@ -175,7 +177,7 @@ public class EvaluateJsonPath extends AbstractProcessor { returnType = destination.equals(DESTINATION_CONTENT) ? RETURN_TYPE_JSON : RETURN_TYPE_SCALAR; } - final DocumentContext documentContext = JsonUtils.validateAndEstablishJsonContext(processSession, flowFile); + final DocumentContext documentContext = validateAndEstablishJsonContext(processSession, flowFile); if (documentContext == null) { logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile}); @@ -194,7 +196,7 @@ public class EvaluateJsonPath extends AbstractProcessor { final ObjectHolder resultHolder = new ObjectHolder<>(null); try { Object result = documentContext.read(jsonPathExp); - if (returnType.equals(RETURN_TYPE_SCALAR) && !JsonUtils.isJsonScalar(result)) { + if (returnType.equals(RETURN_TYPE_SCALAR) && !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); @@ -212,7 +214,7 @@ public class EvaluateJsonPath extends AbstractProcessor { } } - final String resultRepresentation = JsonUtils.getResultRepresentation(resultHolder.get()); + final String resultRepresentation = getResultRepresentation(resultHolder.get()); switch (destination) { case DESTINATION_ATTRIBUTE: jsonPathResults.put(jsonPathAttrKey, resultRepresentation); @@ -232,4 +234,5 @@ public class EvaluateJsonPath extends AbstractProcessor { processSession.transfer(flowFile, REL_MATCH); } + } 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 78e1b2a950..8df2de0a2c 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 @@ -27,9 +27,11 @@ 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.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.processors.standard.util.JsonUtils; import java.io.IOException; import java.io.OutputStream; @@ -44,13 +46,13 @@ import java.util.*; + "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 class SplitJson extends AbstractJsonPathProcessor { 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) + .addValidator(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(); @@ -93,7 +95,7 @@ public class SplitJson extends AbstractProcessor { final ProcessorLog logger = getLogger(); - final DocumentContext documentContext = JsonUtils.validateAndEstablishJsonContext(processSession, original); + final DocumentContext documentContext = validateAndEstablishJsonContext(processSession, original); if (documentContext == null) { logger.error("FlowFile {} did not have valid JSON content.", new Object[]{original}); @@ -129,7 +131,7 @@ public class SplitJson extends AbstractProcessor { split = processSession.write(split, new OutputStreamCallback() { @Override public void process(OutputStream out) throws IOException { - String resultSegmentContent = JsonUtils.getResultRepresentation(resultSegment); + String resultSegmentContent = getResultRepresentation(resultSegment); out.write(resultSegmentContent.getBytes(StandardCharsets.UTF_8)); } }); 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 2174c1ef7e..68b18b8ff5 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 @@ -16,79 +16,19 @@ */ 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; -import java.util.List; -import java.util.Map; /** - * Provides utilities for interacting with JSON elements and JsonPath expressions and results + * Provides utilities for interacting with JSON elements * * @see http://json.org - * @see https://github.com/jayway/JsonPath */ public class JsonUtils { - 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 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(); - } - /** * 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. @@ -104,24 +44,4 @@ public class JsonUtils { } } - /** - * 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, JsonSmartJsonProvider, a Map or List is able to be handled as a JSON entity - 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); - } - } From d6948601cd26d5a7c6cfccaaef297cebe91a9d2e Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Mon, 23 Feb 2015 22:51:37 -0500 Subject: [PATCH 31/43] Adjusting scope of methods as they are specific to the JsonPath related processors --- .../nifi/processors/standard/AbstractJsonPathProcessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index cb4dcf6ab2..54edc701c7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -61,7 +61,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { } }; - public static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) { + static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) { final BooleanHolder validJsonHolder = new BooleanHolder(false); processSession.read(flowFile, new InputStreamCallback() { @@ -101,7 +101,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { return !(obj instanceof Map || obj instanceof List); } - public static String getResultRepresentation(Object jsonPathResult) { + static String getResultRepresentation(Object jsonPathResult) { if (isJsonScalar(jsonPathResult)) { return jsonPathResult.toString(); } From caed7f8468bfee4db47c892bd17d2e36908fc8d2 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 28 Feb 2015 12:36:30 -0500 Subject: [PATCH 32/43] Adding missing subject property on the JsonPath validator. --- .../nifi/processors/standard/AbstractJsonPathProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index 54edc701c7..70efe38fc4 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -57,7 +57,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { } catch (InvalidPathException ipe) { error = ipe.toString(); } - return new ValidationResult.Builder().valid(error == null).explanation(error).build(); + return new ValidationResult.Builder().subject("JsonPath expression " + subject).valid(error == null).explanation(error).build(); } }; From 6a89745ec8646acfdacbf27511cb992c821c51b9 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 28 Feb 2015 15:54:52 -0500 Subject: [PATCH 33/43] Adding a provenance event for EvaluateJsonPath when content is overwritten with selected expression. --- .../org/apache/nifi/processors/standard/EvaluateJsonPath.java | 2 ++ 1 file changed, 2 insertions(+) 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 f7caa68f03..ed0f07c12d 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 @@ -227,6 +227,8 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { } } }); + processSession.getProvenanceReporter().modifyContent(flowFile, + "Replaced content with result of expression " + jsonPathExp.getPath()); break; } } From 57aa5dd63f25c281ba95869bfbfab465465b29f8 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 28 Feb 2015 16:04:43 -0500 Subject: [PATCH 34/43] Providing provenance fork event for the created segments generated by SplitJson. --- .../java/org/apache/nifi/processors/standard/SplitJson.java | 2 ++ 1 file changed, 2 insertions(+) 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 8df2de0a2c..fe2216a7b6 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 @@ -138,6 +138,8 @@ public class SplitJson extends AbstractJsonPathProcessor { segments.add(split); } + processSession.getProvenanceReporter().fork(original, segments); + processSession.transfer(segments, REL_SPLIT); processSession.transfer(original, REL_ORIGINAL); logger.info("Split {} into {} FlowFiles", new Object[]{original, segments.size()}); From 162f02b12fd7b6d1a78efab09942ceffea49e07d Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sat, 28 Feb 2015 21:24:25 -0500 Subject: [PATCH 35/43] Removing the separate reads for validation preferring to do the read once and handle any exceptions. --- .../standard/AbstractJsonPathProcessor.java | 36 ++++++++----------- .../processors/standard/EvaluateJsonPath.java | 15 ++++---- .../nifi/processors/standard/SplitJson.java | 9 ++--- 3 files changed, 26 insertions(+), 34 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index 70efe38fc4..02547f3aba 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -20,7 +20,9 @@ import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.InvalidPathException; import com.jayway.jsonpath.JsonPath; +import com.jayway.jsonpath.internal.spi.json.JsonSmartJsonProvider; import com.jayway.jsonpath.spi.json.JsonProvider; +import net.minidev.json.parser.JSONParser; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; @@ -28,9 +30,7 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.io.InputStreamCallback; -import org.apache.nifi.processors.standard.util.JsonUtils; import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.util.BooleanHolder; import org.apache.nifi.util.ObjectHolder; import java.io.IOException; @@ -46,7 +46,10 @@ import java.util.Map; */ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { - protected static final JsonProvider JSON_PROVIDER = Configuration.defaultConfiguration().jsonProvider(); + private static final Configuration STRICT_PROVIDER_CONFIGURATION = + Configuration.builder().jsonProvider(new JsonSmartJsonProvider(JSONParser.MODE_RFC4627)).build(); + + private static final JsonProvider JSON_PROVIDER = STRICT_PROVIDER_CONFIGURATION.jsonProvider(); public static final Validator JSON_PATH_VALIDATOR = new Validator() { @Override @@ -57,35 +60,23 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { } catch (InvalidPathException ipe) { error = ipe.toString(); } - return new ValidationResult.Builder().subject("JsonPath expression " + subject).valid(error == null).explanation(error).build(); + return new ValidationResult.Builder().subject(subject).valid(error == null).explanation(error).build(); } }; static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) { - - final BooleanHolder validJsonHolder = new BooleanHolder(false); + // Parse the document once into an associated context to support multiple path evaluations if specified + final ObjectHolder contextHolder = new ObjectHolder<>(null); processSession.read(flowFile, new InputStreamCallback() { @Override public void process(InputStream in) throws IOException { - validJsonHolder.set(JsonUtils.isValidJson(in)); + try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) { + DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(bufferedInputStream); + contextHolder.set(ctx); + } } }); - // Parse the document once into an associated context to support multiple path evaluations if specified - final ObjectHolder 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(); } @@ -107,4 +98,5 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { } return JSON_PROVIDER.toJson(jsonPathResult); } + } 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 ed0f07c12d..1b89dee840 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 @@ -17,6 +17,7 @@ package org.apache.nifi.processors.standard; import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.InvalidJsonException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.PathNotFoundException; import org.apache.nifi.annotation.behavior.EventDriven; @@ -177,9 +178,10 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { returnType = destination.equals(DESTINATION_CONTENT) ? RETURN_TYPE_JSON : RETURN_TYPE_SCALAR; } - final DocumentContext documentContext = validateAndEstablishJsonContext(processSession, flowFile); - - if (documentContext == null) { + DocumentContext documentContext = null; + try { + documentContext = validateAndEstablishJsonContext(processSession, flowFile); + } catch (InvalidJsonException e) { logger.error("FlowFile {} did not have valid JSON content.", new Object[]{flowFile}); processSession.transfer(flowFile, REL_FAILURE); return; @@ -187,7 +189,6 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { final Map jsonPathResults = new HashMap<>(); - jsonPathEvalLoop: for (final Map.Entry attributeJsonPathEntry : attributeToJsonPathMap.entrySet()) { String jsonPathAttrKey = attributeJsonPathEntry.getKey(); @@ -207,7 +208,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { 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, StringUtils.EMPTY); - continue jsonPathEvalLoop; + continue; } else { processSession.transfer(flowFile, REL_NO_MATCH); return; @@ -235,6 +236,4 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { flowFile = processSession.putAllAttributes(flowFile, jsonPathResults); processSession.transfer(flowFile, REL_MATCH); } - - -} +} \ No newline at end of file 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 fe2216a7b6..5177bddadc 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 @@ -17,6 +17,7 @@ package org.apache.nifi.processors.standard; import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.InvalidJsonException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.PathNotFoundException; import org.apache.nifi.annotation.behavior.EventDriven; @@ -94,10 +95,10 @@ public class SplitJson extends AbstractJsonPathProcessor { final ProcessorLog logger = getLogger(); - - final DocumentContext documentContext = validateAndEstablishJsonContext(processSession, original); - - if (documentContext == null) { + DocumentContext documentContext = null; + try { + documentContext = validateAndEstablishJsonContext(processSession, original); + } catch (InvalidJsonException e) { logger.error("FlowFile {} did not have valid JSON content.", new Object[]{original}); processSession.transfer(original, REL_FAILURE); return; From b1f971335a464f8de34d4f360327f1c9e5e02bbe Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 00:13:22 -0500 Subject: [PATCH 36/43] Adding processor documentation for EvaluateJsonPath and SplitJson --- .../index.html | 150 ++++++++++++++++++ .../index.html | 79 +++++++++ 2 files changed, 229 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html new file mode 100644 index 0000000000..055a794b01 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html @@ -0,0 +1,150 @@ + + + + + + EvaluateJsonPath + + + + + +

Description:

+ +

+ 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.' + +

+ +

+ Properties: +

+ +

+ In the list below, the names of required properties appear in bold. + Any other properties (not in bold) are considered optional. If a + property has a default value, it is indicated. If a property + supports the use of the NiFi Expression Language (or simply, + "expression language"), that is also indicated. +

+ +

+ Modifies Attributes: +

+ +

+ This processor adds user-defined attributes if the <Destination> property is set to + flowfile-attribute. +

+ + +
    +
  • + Destination +
      +
    • Indicates whether the results of the JsonPath evaluation are + written to the FlowFile content or a FlowFile attribute; if using + attribute, the attribute's name must be specified in the value of + the Attribute Name property. +
    • +
    • + Valid values are: +
        +
      • flowfile-content
      • +
      • flowfile-attribute
      • +
      +
    • +
    • Default value: flowfile-content
    • +
    • Supports expression language: false
    • +
    +
  • +
  • + Return Type +
      +
    • Indicates the desired return type of the Xpath expressions. + Selecting 'auto-detect' will set the return type to 'json' for a + Destination of 'flowfile-content', and 'scalar' for a Destination of + 'flowfile-attribute'.") +
    • +
    • + Valid values are: +
        +
      • auto-detect
      • +
      • json
      • +
      • scalar
      • +
      +
    • +
    • Default value: auto-detect
    • +
    • Supports expression language: false
    • +
    +
  • +
  • + user-defined properties +
      +
    • The name of the attribute to put the JsonPath result into if + flowfile-attribute is used as the value for the Destination + property; if using flowfile-content as the value for the + Destination property, this value is ignored. +
    • +
    • Supports expression language: false
    • +
    +
  • +
+ +

+ Relationships: +

+
    +
  • + failure +
      +
    • If the JsonPath cannot be evaluated against the content of the + FlowFile, then the FlowFile follows this relationship. For + example, if the FlowFile does not contain valid JSON. +
    • +
    +
  • +
  • + matched +
      +
    • If the JsonPath is successfully evaluated and the FlowFile is + modified as a result, then the FlowFile follows this + relationship. +
    • +
    +
  • +
  • + unmatched +
      +
    • If the JsonPath does not match the content of the FlowFile, then + the FlowFile follows this relationship. +
    • +
    +
  • +
+

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html new file mode 100644 index 0000000000..6fc8e6145a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html @@ -0,0 +1,79 @@ + + + + + + SplitJson + + + + + + +

Description:

+ +

+ This processor 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. +

+ +Properties: +

+ +

+ In the list below, the names of required properties appear in bold. Any other properties (not in bold) are + considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi + Expression Language (or simply, "expression language"), that is also indicated. +

+
    +
  • JsonPath Expression +
      +
    • A JsonPath expression that indicates the array element to split into JSON/scalar fragments.
    • +
    • Supports expression language: false
    • +
    +
  • +
+ +

+ Relationships: +

+
    +
  • failure +
      +
    • If a FlowFile fails processing for any reason (for example, the FlowFile is not valid JSON or the + specified path does not exist) and does not go down the original relationship. +
    • +
    +
  • +
  • original +
      +
    • If FlowFiles are successfully split, a copy of the original FlowFile follows this relationship.
    • +
    +
  • +
  • split +
      +
    • If FlowFiles are successfully split into one or more files, those split files follow this + relationship. +
    • +
    +
  • +
+ + + From 4d3cff3592d16d1ce5608b20a025edf34a7c69d7 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 13:26:03 -0500 Subject: [PATCH 37/43] Removing JsonUtils as all functionality was migrated into AbstractJsonPathProcessor given its limited utility outside of those classes. Adjusting validation approach for JsonPath processors to accomodate caching of expressions. --- .../standard/AbstractJsonPathProcessor.java | 41 +++++++++++----- .../processors/standard/EvaluateJsonPath.java | 43 ++++++++++++++++- .../nifi/processors/standard/SplitJson.java | 32 +++++++++++-- .../processors/standard/util/JsonUtils.java | 47 ------------------- 4 files changed, 98 insertions(+), 65 deletions(-) delete mode 100644 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/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index 02547f3aba..baeef7b73b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -51,19 +51,6 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { private static final JsonProvider JSON_PROVIDER = STRICT_PROVIDER_CONFIGURATION.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().subject(subject).valid(error == null).explanation(error).build(); - } - }; - static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) { // Parse the document once into an associated context to support multiple path evaluations if specified final ObjectHolder contextHolder = new ObjectHolder<>(null); @@ -99,4 +86,32 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { return JSON_PROVIDER.toJson(jsonPathResult); } + protected abstract static class JsonPathValidator implements Validator { + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + JsonPath compiledJsonPath = null; + String error = null; + try { + if (isStale(subject, input)) { + compiledJsonPath = JsonPath.compile(input); + cacheComputedValue(subject, input, compiledJsonPath); + } + } catch (InvalidPathException ipe) { + error = ipe.toString(); + } + return new ValidationResult.Builder().subject(subject).valid(error == null).explanation(error).build(); + } + + /** + * An optional hook to act on the compute value + */ + abstract void cacheComputedValue(String subject, String input, JsonPath computedJsonPath); + + /** + * A hook for implementing classes to determine if a cached value is stale for a compiled JsonPath represented + * by either a validation + */ + abstract boolean isStale(String subject, String input); + } } 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 1b89dee840..b40f6c6da5 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 @@ -20,11 +20,13 @@ import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.InvalidJsonException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.PathNotFoundException; +import org.apache.commons.lang3.StringUtils; 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.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -38,12 +40,14 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.util.ObjectHolder; -import org.apache.nifi.util.StringUtils; +import org.apache.nifi.util.Tuple; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; @EventDriven @SideEffectFree @@ -92,6 +96,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { private Set relationships; private List properties; + private ConcurrentMap> cachedJsonPathMap = new ConcurrentHashMap<>(); @Override protected void init(final ProcessorInitializationContext context) { @@ -145,12 +150,46 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { return new PropertyDescriptor.Builder() .name(propertyDescriptorName) .expressionLanguageSupported(false) - .addValidator(JSON_PATH_VALIDATOR) + .addValidator(new JsonPathValidator() { + @Override + public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) { + cachedJsonPathMap.put(subject, new Tuple<>(input, computedJsonPath)); + + } + + @Override + public boolean isStale(String subject, String input) { + return cachedJsonPathMap.get(subject) == null; + } + }) .required(false) .dynamic(true) .build(); } + @Override + public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + if (descriptor.isDynamic()) { + if (!StringUtils.equals(oldValue, newValue)) { + cachedJsonPathMap.remove(descriptor.getName()); + } + } + } + + /** + * Provides cleanup of the map for any JsonPath values that may have been created. This will remove common values + * shared between multiple instances, but will be regenerated when the next validation cycle occurs as a result of + * isStale() + */ + @OnRemoved + public void onRemoved() { + for (PropertyDescriptor propertyDescriptor : getPropertyDescriptors()) { + if (propertyDescriptor.isDynamic()) { + cachedJsonPathMap.remove(propertyDescriptor.getName()); + } + } + } + @Override public void onTrigger(ProcessContext processContext, final ProcessSession processSession) throws ProcessException { 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 5177bddadc..8c7ae4dba7 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 @@ -20,6 +20,7 @@ import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.InvalidJsonException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.PathNotFoundException; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; @@ -38,6 +39,8 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; @EventDriven @SideEffectFree @@ -53,7 +56,17 @@ public class SplitJson extends AbstractJsonPathProcessor { .name("JsonPath Expression") .description("A JsonPath expression that indicates the array element to split into JSON/scalar fragments.") .required(true) - .addValidator(JSON_PATH_VALIDATOR) + .addValidator(new JsonPathValidator() { + @Override + public void cacheComputedValue(String subject, String input, JsonPath computedJson) { + JSON_PATH_MAP.put(input, computedJson); + } + + @Override + public boolean isStale(String subject, String input) { + return JSON_PATH_MAP.get(input) == null; + } + }) .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(); @@ -63,6 +76,8 @@ public class SplitJson extends AbstractJsonPathProcessor { private List properties; private Set relationships; + private static final ConcurrentMap JSON_PATH_MAP = new ConcurrentHashMap(); + @Override protected void init(final ProcessorInitializationContext context) { final List properties = new ArrayList<>(); @@ -86,6 +101,16 @@ public class SplitJson extends AbstractJsonPathProcessor { return properties; } + @Override + public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + if (descriptor.equals(ARRAY_JSON_PATH_EXPRESSION)) { + if (!StringUtils.equals(oldValue, newValue)) { + // clear the cached item + JSON_PATH_MAP.remove(oldValue); + } + } + } + @Override public void onTrigger(final ProcessContext processContext, final ProcessSession processSession) { final FlowFile original = processSession.get(); @@ -104,8 +129,9 @@ public class SplitJson extends AbstractJsonPathProcessor { return; } - final String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); - final JsonPath jsonPath = JsonPath.compile(jsonPathExpression); + String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); + final JsonPath jsonPath = JSON_PATH_MAP.get(jsonPathExpression); + getLogger().info("Using value {} for split ", new Object[]{jsonPathExpression}); final List segments = new ArrayList<>(); 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 deleted file mode 100644 index 68b18b8ff5..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonUtils.java +++ /dev/null @@ -1,47 +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 net.minidev.json.JSONValue; - -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; - -/** - * Provides utilities for interacting with JSON elements - * - * @see http://json.org - */ -public class JsonUtils { - - /** - * 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 { - try (InputStreamReader inputStreamReader = new InputStreamReader(inputStream)) { - return JSONValue.isValidJsonStrict(inputStreamReader); - } - } - -} From 84602ca3e9935da74788aa893998efd2685dc873 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 13:27:40 -0500 Subject: [PATCH 38/43] Removing extraneous logging statement. --- .../main/java/org/apache/nifi/processors/standard/SplitJson.java | 1 - 1 file changed, 1 deletion(-) 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 8c7ae4dba7..59f4d71794 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 @@ -131,7 +131,6 @@ public class SplitJson extends AbstractJsonPathProcessor { String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); final JsonPath jsonPath = JSON_PATH_MAP.get(jsonPathExpression); - getLogger().info("Using value {} for split ", new Object[]{jsonPathExpression}); final List segments = new ArrayList<>(); From 5a2a8fc6befb8305407a3f9b90443b66237e14f5 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 13:37:29 -0500 Subject: [PATCH 39/43] Adding notes about JsonPath loading contents into memory for both JsonPath processors. --- .../index.html | 5 +++++ .../index.html | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html index 055a794b01..58da666403 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html @@ -36,7 +36,12 @@ 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.' +

+

+ Note: The underlying JsonPath library loads the entirety of the streamed content into and performs + result evaluations in memory. Accordingly, it is important to consider the anticipated profile of content being + evaluated by this processor and the hardware supporting it especially when working against large JSON documents.

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html index 6fc8e6145a..e3a4b14e66 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html @@ -33,6 +33,12 @@ does not evaluate to an array element, the original file is routed to 'failure' and no files are generated.

+

+ Note: The underlying JsonPath library loads the entirety of the streamed content into and performs + result evaluations in memory. Accordingly, it is important to consider the anticipated profile of content being + evaluated by this processor and the hardware supporting it especially when working against large JSON documents. +

+ Properties:

From 484687a67b12fb5fc13dabd7851f83a6e4a898be Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 14:16:22 -0500 Subject: [PATCH 40/43] Adjusting onRemoved methods for both JsonPath processors to clean up entries on exit. --- .../standard/AbstractJsonPathProcessor.java | 2 +- .../nifi/processors/standard/EvaluateJsonPath.java | 2 +- .../apache/nifi/processors/standard/SplitJson.java | 12 ++++++++++++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index baeef7b73b..febc3f8b66 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -86,7 +86,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { return JSON_PROVIDER.toJson(jsonPathResult); } - protected abstract static class JsonPathValidator implements Validator { + abstract static class JsonPathValidator implements Validator { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { 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 b40f6c6da5..81c9bbec27 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 @@ -182,7 +182,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { * isStale() */ @OnRemoved - public void onRemoved() { + public void onRemoved(ProcessContext processContext) { for (PropertyDescriptor propertyDescriptor : getPropertyDescriptors()) { if (propertyDescriptor.isDynamic()) { cachedJsonPathMap.remove(propertyDescriptor.getName()); 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 59f4d71794..7bb8c4e12c 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 @@ -26,6 +26,7 @@ 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.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; @@ -111,6 +112,17 @@ public class SplitJson extends AbstractJsonPathProcessor { } } + /** + * Provides cleanup of the map for any JsonPath values that may have been created. This will remove common values + * shared between multiple instances, but will be regenerated when the next validation cycle occurs as a result of + * isStale() + */ + @OnRemoved + public void onRemoved(ProcessContext processContext) { + String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); + JSON_PATH_MAP.remove(jsonPathExpression); + } + @Override public void onTrigger(final ProcessContext processContext, final ProcessSession processSession) { final FlowFile original = processSession.get(); From 973b493386c71017f9baa233d4ec178251e64f53 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 16:31:32 -0500 Subject: [PATCH 41/43] Adjusting handling of map to cache data items on an instance basis. --- .../processors/standard/EvaluateJsonPath.java | 14 ++--- .../nifi/processors/standard/SplitJson.java | 55 +++++++++---------- 2 files changed, 34 insertions(+), 35 deletions(-) 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 81c9bbec27..64f6e0d428 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 @@ -40,7 +40,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.util.ObjectHolder; -import org.apache.nifi.util.Tuple; import java.io.IOException; import java.io.OutputStream; @@ -96,7 +95,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { private Set relationships; private List properties; - private ConcurrentMap> cachedJsonPathMap = new ConcurrentHashMap<>(); + private final ConcurrentMap cachedJsonPathMap = new ConcurrentHashMap<>(); @Override protected void init(final ProcessorInitializationContext context) { @@ -153,13 +152,12 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { .addValidator(new JsonPathValidator() { @Override public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) { - cachedJsonPathMap.put(subject, new Tuple<>(input, computedJsonPath)); - + cachedJsonPathMap.put(input, computedJsonPath); } @Override public boolean isStale(String subject, String input) { - return cachedJsonPathMap.get(subject) == null; + return cachedJsonPathMap.get(input) == null; } }) .required(false) @@ -171,7 +169,9 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { if (descriptor.isDynamic()) { if (!StringUtils.equals(oldValue, newValue)) { - cachedJsonPathMap.remove(descriptor.getName()); + if (oldValue != null) { + cachedJsonPathMap.remove(oldValue); + } } } } @@ -185,7 +185,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor { public void onRemoved(ProcessContext processContext) { for (PropertyDescriptor propertyDescriptor : getPropertyDescriptors()) { if (propertyDescriptor.isDynamic()) { - cachedJsonPathMap.remove(propertyDescriptor.getName()); + cachedJsonPathMap.remove(processContext.getProperty(propertyDescriptor).getValue()); } } } 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 7bb8c4e12c..5a193a15b8 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 @@ -26,8 +26,9 @@ 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.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.ProcessContext; @@ -35,13 +36,13 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.*; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; @EventDriven @SideEffectFree @@ -56,18 +57,8 @@ public class SplitJson extends AbstractJsonPathProcessor { 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.") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(true) - .addValidator(new JsonPathValidator() { - @Override - public void cacheComputedValue(String subject, String input, JsonPath computedJson) { - JSON_PATH_MAP.put(input, computedJson); - } - - @Override - public boolean isStale(String subject, String input) { - return JSON_PATH_MAP.get(input) == null; - } - }) .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(); @@ -77,7 +68,7 @@ public class SplitJson extends AbstractJsonPathProcessor { private List properties; private Set relationships; - private static final ConcurrentMap JSON_PATH_MAP = new ConcurrentHashMap(); + private final AtomicReference JSON_PATH_REF = new AtomicReference<>(); @Override protected void init(final ProcessorInitializationContext context) { @@ -106,21 +97,30 @@ public class SplitJson extends AbstractJsonPathProcessor { public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { if (descriptor.equals(ARRAY_JSON_PATH_EXPRESSION)) { if (!StringUtils.equals(oldValue, newValue)) { - // clear the cached item - JSON_PATH_MAP.remove(oldValue); + if (oldValue != null) { + // clear the cached item + JSON_PATH_REF.set(null); + } } } } - /** - * Provides cleanup of the map for any JsonPath values that may have been created. This will remove common values - * shared between multiple instances, but will be regenerated when the next validation cycle occurs as a result of - * isStale() - */ - @OnRemoved - public void onRemoved(ProcessContext processContext) { - String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); - JSON_PATH_MAP.remove(jsonPathExpression); + @Override + protected Collection customValidate(ValidationContext validationContext) { + JsonPathValidator validator = new JsonPathValidator() { + @Override + public void cacheComputedValue(String subject, String input, JsonPath computedJson) { + JSON_PATH_REF.set(computedJson); + } + + @Override + public boolean isStale(String subject, String input) { + return JSON_PATH_REF.get() == null; + } + }; + + String value = validationContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); + return Collections.singleton(validator.validate(ARRAY_JSON_PATH_EXPRESSION.getName(), value, validationContext)); } @Override @@ -141,8 +141,7 @@ public class SplitJson extends AbstractJsonPathProcessor { return; } - String jsonPathExpression = processContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue(); - final JsonPath jsonPath = JSON_PATH_MAP.get(jsonPathExpression); + final JsonPath jsonPath = JSON_PATH_REF.get(); final List segments = new ArrayList<>(); From 4618f46f278c50238ec98bf7021337564e641de0 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 21:23:01 -0500 Subject: [PATCH 42/43] Adding JsonPathExpressionValidator to perform an exception free validation of JsonPath expressions. This is used as a screen before attempting a compile. --- .../standard/AbstractJsonPathProcessor.java | 14 +- .../nifi/processors/standard/SplitJson.java | 2 +- .../util/JsonPathExpressionValidator.java | 487 ++++++++++++++++++ .../standard/TestEvaluateJsonPath.java | 12 + 4 files changed, 507 insertions(+), 8 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathExpressionValidator.java diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index febc3f8b66..94a299ea37 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -18,7 +18,6 @@ package org.apache.nifi.processors.standard; import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.InvalidPathException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.internal.spi.json.JsonSmartJsonProvider; import com.jayway.jsonpath.spi.json.JsonProvider; @@ -30,6 +29,7 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processors.standard.util.JsonPathExpressionValidator; import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.util.ObjectHolder; @@ -90,15 +90,14 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { - JsonPath compiledJsonPath = null; String error = null; - try { - if (isStale(subject, input)) { - compiledJsonPath = JsonPath.compile(input); + if (isStale(subject, input)) { + if (JsonPathExpressionValidator.isValidExpression(input)) { + JsonPath compiledJsonPath = JsonPath.compile(input); cacheComputedValue(subject, input, compiledJsonPath); + } else { + error = "specified expression was not valid: " + input; } - } catch (InvalidPathException ipe) { - error = ipe.toString(); } return new ValidationResult.Builder().subject(subject).valid(error == null).explanation(error).build(); } @@ -106,6 +105,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { /** * An optional hook to act on the compute value */ + abstract void cacheComputedValue(String subject, String input, JsonPath computedJsonPath); /** 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 5a193a15b8..4d79746db9 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 @@ -57,7 +57,7 @@ public class SplitJson extends AbstractJsonPathProcessor { 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.") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) // Full validation/caching occurs in #customValidate .required(true) .build(); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathExpressionValidator.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathExpressionValidator.java new file mode 100644 index 0000000000..61f9bbe2e0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JsonPathExpressionValidator.java @@ -0,0 +1,487 @@ +/* + * 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.Filter; +import com.jayway.jsonpath.Predicate; +import com.jayway.jsonpath.internal.Utils; +import com.jayway.jsonpath.internal.token.*; +import org.apache.nifi.util.StringUtils; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.regex.Pattern; + +import static java.util.Arrays.asList; + +/** + * JsonPathExpressionValidator performs the same execution as com.jayway.jsonpath.internal.PathCompiler, but does not throw + * exceptions when an invalid path segment is found. + * Limited access to create JsonPath objects requires a separate flow of execution in avoiding exceptions. + * + * @see https://github.com/jayway/JsonPath + */ +public class JsonPathExpressionValidator { + + private static final String PROPERTY_OPEN = "['"; + private static final String PROPERTY_CLOSE = "']"; + private static final char DOCUMENT = '$'; + private static final char ANY = '*'; + private static final char PERIOD = '.'; + private static final char BRACKET_OPEN = '['; + private static final char BRACKET_CLOSE = ']'; + private static final char SPACE = ' '; + + + /** + * Performs a validation of a provided JsonPath expression. + *

+ * Typically this is used in the context of: + * + *

+     * JsonPath compiledJsonPath = null;
+     * if (JsonPathExpressionValidator.isValidExpression(input)) {
+     *      compiledJsonPath = JsonPath.compile(input);
+     *      ...
+     * } else {
+     *      // error handling
+     * }
+     * 
+ * + * + * @param path to evaluate for validity + * @param filters applied to path expression; this is typically unused in the context of Processors + * @return true if the specified path is valid; false otherwise + */ + public static boolean isValidExpression(String path, Predicate... filters) { + path = path.trim(); + if (StringUtils.isBlank(path)) { + // "Path may not be null empty" + return false; + } + if (path.endsWith("..")) { + // "A path can not end with a scan." + return false; + } + + LinkedList filterList = new LinkedList(asList(filters)); + + if (path.charAt(0) != '$' && path.charAt(0) != '@') { + path = "$." + path; + } + + if (path.charAt(0) == '@') { + path = "$" + path.substring(1); + } + + if (path.length() > 1 && path.charAt(1) != '.' && path.charAt(1) != '[') { + // "Invalid path " + path + return false; + } + + RootPathToken root = null; + + int i = 0; + int positions; + String fragment = ""; + + do { + char current = path.charAt(i); + + switch (current) { + case SPACE: + // "Space not allowed in path" + return false; + case DOCUMENT: + fragment = "$"; + i++; + break; + case BRACKET_OPEN: + positions = fastForwardUntilClosed(path, i); + fragment = path.substring(i, i + positions); + i += positions; + break; + case PERIOD: + i++; + if (path.charAt(i) == PERIOD) { + //This is a deep scan + fragment = ".."; + i++; + } else { + positions = fastForward(path, i); + if (positions == 0) { + continue; + + } else if (positions == 1 && path.charAt(i) == '*') { + fragment = new String("[*]"); + } else { + fragment = PROPERTY_OPEN + path.substring(i, i + positions) + PROPERTY_CLOSE; + } + i += positions; + } + break; + case ANY: + fragment = new String("[*]"); + i++; + break; + default: + positions = fastForward(path, i); + + fragment = PROPERTY_OPEN + path.substring(i, i + positions) + PROPERTY_CLOSE; + i += positions; + break; + } + + /* + * Analyze each component represented by a fragment. If there is a failure to properly evaluate, + * a null result is returned + */ + PathToken analyzedComponent = PathComponentAnalyzer.analyze(fragment, filterList); + if (analyzedComponent == null) { + return false; + } + + if (root == null) { + root = (RootPathToken) analyzedComponent; + } else { + root.append(analyzedComponent); + } + + + } while (i < path.length()); + + return true; + } + + private static int fastForward(String s, int index) { + int skipCount = 0; + while (index < s.length()) { + char current = s.charAt(index); + if (current == PERIOD || current == BRACKET_OPEN || current == SPACE) { + break; + } + index++; + skipCount++; + } + return skipCount; + } + + private static int fastForwardUntilClosed(String s, int index) { + int skipCount = 0; + int nestedBrackets = 0; + + //First char is always '[' no need to check it + index++; + skipCount++; + + while (index < s.length()) { + char current = s.charAt(index); + + index++; + skipCount++; + + if (current == BRACKET_CLOSE && nestedBrackets == 0) { + break; + } + if (current == BRACKET_OPEN) { + nestedBrackets++; + } + if (current == BRACKET_CLOSE) { + nestedBrackets--; + } + } + return skipCount; + } + + static class PathComponentAnalyzer { + + private static final Pattern FILTER_PATTERN = Pattern.compile("^\\[\\s*\\?\\s*[,\\s*\\?]*?\\s*]$"); //[?] or [?, ?, ...] + private int i; + private char current; + + private final LinkedList filterList; + private final String pathFragment; + + PathComponentAnalyzer(String pathFragment, LinkedList filterList) { + this.pathFragment = pathFragment; + this.filterList = filterList; + } + + static PathToken analyze(String pathFragment, LinkedList filterList) { + return new PathComponentAnalyzer(pathFragment, filterList).analyze(); + } + + public PathToken analyze() { + + if ("$".equals(pathFragment)) return new RootPathToken(); + else if ("..".equals(pathFragment)) return new ScanPathToken(); + else if ("[*]".equals(pathFragment)) return new WildcardPathToken(); + else if (".*".equals(pathFragment)) return new WildcardPathToken(); + else if ("[?]".equals(pathFragment)) return new PredicatePathToken(filterList.poll()); + + else if (FILTER_PATTERN.matcher(pathFragment).matches()) { + final int criteriaCount = Utils.countMatches(pathFragment, "?"); + List filters = new ArrayList<>(criteriaCount); + for (int i = 0; i < criteriaCount; i++) { + filters.add(filterList.poll()); + } + return new PredicatePathToken(filters); + } + + this.i = 0; + do { + current = pathFragment.charAt(i); + + switch (current) { + case '?': + return analyzeCriteriaSequence4(); + case '\'': + return analyzeProperty(); + default: + if (Character.isDigit(current) || current == ':' || current == '-' || current == '@') { + return analyzeArraySequence(); + } + i++; + break; + } + + + } while (i < pathFragment.length()); + + //"Could not analyze path component: " + pathFragment + return null; + } + + + public PathToken analyzeCriteriaSequence4() { + int[] bounds = findFilterBounds(); + if (bounds == null) { + return null; + } + i = bounds[1]; + + return new PredicatePathToken(Filter.parse(pathFragment.substring(bounds[0], bounds[1]))); + } + + int[] findFilterBounds() { + int end = 0; + int start = i; + + while (pathFragment.charAt(start) != '[') { + start--; + } + + int mem = ' '; + int curr = start; + boolean inProp = false; + int openSquareBracket = 0; + int openBrackets = 0; + while (end == 0) { + char c = pathFragment.charAt(curr); + switch (c) { + case '(': + if (!inProp) openBrackets++; + break; + case ')': + if (!inProp) openBrackets--; + break; + case '[': + if (!inProp) openSquareBracket++; + break; + case ']': + if (!inProp) { + openSquareBracket--; + if (openBrackets == 0) { + end = curr + 1; + } + } + break; + case '\'': + if (mem == '\\') { + break; + } + inProp = !inProp; + break; + default: + break; + } + mem = c; + curr++; + } + if (openBrackets != 0 || openSquareBracket != 0) { + // "Filter brackets are not balanced" + return null; + } + return new int[]{start, end}; + } + + + //"['foo']" + private PathToken analyzeProperty() { + List properties = new ArrayList(); + StringBuilder buffer = new StringBuilder(); + + boolean propertyIsOpen = false; + + while (current != ']') { + switch (current) { + case '\'': + if (propertyIsOpen) { + properties.add(buffer.toString()); + buffer.setLength(0); + propertyIsOpen = false; + } else { + propertyIsOpen = true; + } + break; + default: + if (propertyIsOpen) { + buffer.append(current); + } + break; + } + current = pathFragment.charAt(++i); + } + return new PropertyPathToken(properties); + } + + + //"[-1:]" sliceFrom + //"[:1]" sliceTo + //"[0:5]" sliceBetween + //"[1]" + //"[1,2,3]" + //"[(@.length - 1)]" + private PathToken analyzeArraySequence() { + StringBuilder buffer = new StringBuilder(); + List numbers = new ArrayList(); + + boolean contextSize = (current == '@'); + boolean sliceTo = false; + boolean sliceFrom = false; + boolean sliceBetween = false; + boolean indexSequence = false; + boolean singleIndex = false; + + if (contextSize) { + + current = pathFragment.charAt(++i); + current = pathFragment.charAt(++i); + while (current != '-') { + if (current == ' ' || current == '(' || current == ')') { + current = pathFragment.charAt(++i); + continue; + } + buffer.append(current); + current = pathFragment.charAt(++i); + } + String function = buffer.toString(); + buffer.setLength(0); + if (!function.equals("size") && !function.equals("length")) { + // "Invalid function: @." + function + ". Supported functions are: [(@.length - n)] and [(@.size() - n)]" + return null; + } + while (current != ')') { + if (current == ' ') { + current = pathFragment.charAt(++i); + continue; + } + buffer.append(current); + current = pathFragment.charAt(++i); + } + + } else { + + + while (Character.isDigit(current) || current == ',' || current == ' ' || current == ':' || current == '-') { + + switch (current) { + case ' ': + break; + case ':': + if (buffer.length() == 0) { + //this is a tail slice [:12] + sliceTo = true; + current = pathFragment.charAt(++i); + while (Character.isDigit(current) || current == ' ' || current == '-') { + if (current != ' ') { + buffer.append(current); + } + current = pathFragment.charAt(++i); + } + numbers.add(Integer.parseInt(buffer.toString())); + buffer.setLength(0); + } else { + //we now this starts with [12:??? + numbers.add(Integer.parseInt(buffer.toString())); + buffer.setLength(0); + current = pathFragment.charAt(++i); + + //this is a tail slice [:12] + while (Character.isDigit(current) || current == ' ' || current == '-') { + if (current != ' ') { + buffer.append(current); + } + current = pathFragment.charAt(++i); + } + + if (buffer.length() == 0) { + sliceFrom = true; + } else { + sliceBetween = true; + numbers.add(Integer.parseInt(buffer.toString())); + buffer.setLength(0); + } + } + break; + case ',': + numbers.add(Integer.parseInt(buffer.toString())); + buffer.setLength(0); + indexSequence = true; + break; + default: + buffer.append(current); + break; + } + if (current == ']') { + break; + } + current = pathFragment.charAt(++i); + } + } + if (buffer.length() > 0) { + numbers.add(Integer.parseInt(buffer.toString())); + } + singleIndex = (numbers.size() == 1) && !sliceTo && !sliceFrom && !contextSize; + + ArrayPathToken.Operation operation = null; + + if (singleIndex) operation = ArrayPathToken.Operation.SINGLE_INDEX; + else if (indexSequence) operation = ArrayPathToken.Operation.INDEX_SEQUENCE; + else if (sliceFrom) operation = ArrayPathToken.Operation.SLICE_FROM; + else if (sliceTo) operation = ArrayPathToken.Operation.SLICE_TO; + else if (sliceBetween) operation = ArrayPathToken.Operation.SLICE_BETWEEN; + else if (contextSize) operation = ArrayPathToken.Operation.CONTEXT_SIZE; + + assert operation != null; + + return new ArrayPathToken(numbers, operation); + + } + } +} \ No newline at end of file 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 c5ff81476d..058e21c21d 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 @@ -67,6 +67,18 @@ public class TestEvaluateJsonPath { Assert.fail("Processor incorrectly ran with an invalid configuration of multiple paths specified as attributes for a destination of content."); } + @Test(expected = AssertionError.class) + public void testInvalidConfiguration_invalidJsonPath_space() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateJsonPath()); + testRunner.setProperty(EvaluateJsonPath.DESTINATION, EvaluateJsonPath.DESTINATION_CONTENT); + testRunner.setProperty("JsonPath1", "$[0]. _id"); + + 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()); From e6ebaa4ced457785f33df39c8635056d8a683ac4 Mon Sep 17 00:00:00 2001 From: Aldrin Piri Date: Sun, 1 Mar 2015 21:42:56 -0500 Subject: [PATCH 43/43] Adding licensing and notice information for JsonPath --- nifi/LICENSE | 22 +++++++++++++++++++ nifi/nifi-assembly/NOTICE | 4 ++++ .../src/main/resources/META-INF/NOTICE | 6 ++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/nifi/LICENSE b/nifi/LICENSE index 50150e3537..d00d4508e5 100644 --- a/nifi/LICENSE +++ b/nifi/LICENSE @@ -455,3 +455,25 @@ 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 source from 'JsonPath'. Specifically the 'PathCompiler'. +The source is available under an Apache Software License 2.0. + + Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project + + Permission is hereby granted, free of charge, to any person obtaining a copy + of this software and associated documentation files (the "Software"), to deal + in the Software without restriction, including without limitation the rights + to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + copies of the Software, and to permit persons to whom the Software is + furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + THE SOFTWARE. diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index 1667e9eaa4..025d265138 100644 --- a/nifi/nifi-assembly/NOTICE +++ b/nifi/nifi-assembly/NOTICE @@ -466,6 +466,10 @@ The following binary components are provided under the Apache Software License v The following NOTICE information applies: Copyright 2011 JSON-SMART authors + (ASLv2) JsonPath + The following NOTICE information applies: + Copyright 2011 JsonPath authors + ************************ Common Development and Distribution License 1.1 ************************ 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 d63a74ddb7..18961c5c2a 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 @@ -82,7 +82,11 @@ The following binary components are provided under the Apache Software License v (ASLv2) JSON-SMART The following NOTICE information applies: Copyright 2011 JSON-SMART authors - + + (ASLv2) JsonPath + The following NOTICE information applies: + Copyright 2011 JsonPath authors + ************************ Common Development and Distribution License 1.1 ************************