You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by pv...@apache.org on 2019/08/07 07:42:57 UTC

[nifi] branch master updated: NIFI-6525 - Support JsonPath delete expressions

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

pvillard pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 2491c51  NIFI-6525 - Support JsonPath delete expressions
2491c51 is described below

commit 2491c51d34a4012dd6c9058f60f80ea2203e311b
Author: mans2singh <ma...@yahoo.com>
AuthorDate: Sat Aug 3 13:52:04 2019 -0400

    NIFI-6525 - Support JsonPath delete expressions
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #3632.
---
 .../language/antlr/AttributeExpressionLexer.g      |  1 +
 .../language/antlr/AttributeExpressionParser.g     |  2 +-
 .../language/compile/ExpressionCompiler.java       |  7 ++
 ...thEvaluator.java => JsonPathBaseEvaluator.java} | 72 ++++++++--------
 .../functions/JsonPathDeleteEvaluator.java         | 54 ++++++++++++
 .../evaluation/functions/JsonPathEvaluator.java    | 95 +++-------------------
 .../attribute/expression/language/TestQuery.java   | 63 ++++++++++++++
 .../main/asciidoc/expression-language-guide.adoc   | 52 ++++++++++++
 8 files changed, 221 insertions(+), 125 deletions(-)

diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
index 3b8e65f..576210d 100644
--- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
+++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
@@ -180,6 +180,7 @@ AND : 'and';
 JOIN : 'join';
 TO_LITERAL : 'literal';
 JSON_PATH : 'jsonPath';
+JSON_PATH_DELETE : 'jsonPathDelete';
 
 // 2 arg functions
 SUBSTRING	: 'substring';
diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
index 2728e36..f7e7b00 100644
--- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
+++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
@@ -76,7 +76,7 @@ tokens {
 // functions that return Strings
 zeroArgString : (TO_UPPER | TO_LOWER | TRIM | TO_STRING | URL_ENCODE | URL_DECODE | BASE64_ENCODE | BASE64_DECODE | ESCAPE_JSON | ESCAPE_XML | ESCAPE_CSV | ESCAPE_HTML3 | ESCAPE_HTML4 | UNESCAPE_JSON | UNESCAPE_XML | UNESCAPE_CSV | UNESCAPE_HTML3 | UNESCAPE_HTML4 ) LPAREN! RPAREN!;
 oneArgString : ((SUBSTRING_BEFORE | SUBSTRING_BEFORE_LAST | SUBSTRING_AFTER | SUBSTRING_AFTER_LAST | REPLACE_NULL | REPLACE_EMPTY |
-				PREPEND | APPEND | STARTS_WITH | ENDS_WITH | CONTAINS | JOIN | JSON_PATH | FROM_RADIX) LPAREN! anyArg RPAREN!) |
+				PREPEND | APPEND | STARTS_WITH | ENDS_WITH | CONTAINS | JOIN | JSON_PATH | JSON_PATH_DELETE | FROM_RADIX) LPAREN! anyArg RPAREN!) |
 			   (TO_RADIX LPAREN! anyArg (COMMA! anyArg)? RPAREN!);
 twoArgString : ((REPLACE | REPLACE_FIRST | REPLACE_ALL | IF_ELSE) LPAREN! anyArg COMMA! anyArg RPAREN!) |
 			   ((SUBSTRING | FORMAT) LPAREN! anyArg (COMMA! anyArg)? RPAREN!);
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
index eb258a4..56d4571 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
@@ -61,6 +61,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.functions.InEval
 import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathDeleteEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
@@ -173,6 +174,7 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JOIN;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH_DELETE;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
 import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
@@ -912,6 +914,11 @@ public class ExpressionCompiler {
                 return addToken(new JsonPathEvaluator(toStringEvaluator(subjectEvaluator),
                     toStringEvaluator(argEvaluators.get(0), "first argument to jsonPath")), "jsonPath");
             }
+            case JSON_PATH_DELETE: {
+                verifyArgCount(argEvaluators, 1, "jsonPathDelete");
+                return addToken(new JsonPathDeleteEvaluator(toStringEvaluator(subjectEvaluator),
+                        toStringEvaluator(argEvaluators.get(0), "first argument to jsonPathDelete")), "jsonPathDelete");
+            }
             case IF_ELSE: {
                 verifyArgCount(argEvaluators, 2, "ifElse");
                 return addToken(new IfElseEvaluator(toBooleanEvaluator(subjectEvaluator),
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathBaseEvaluator.java
similarity index 71%
copy from nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
copy to nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathBaseEvaluator.java
index aadb324..e260bd6 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathBaseEvaluator.java
@@ -16,56 +16,57 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.DocumentContext;
-import com.jayway.jsonpath.InvalidJsonException;
-import com.jayway.jsonpath.JsonPath;
-import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
-import com.jayway.jsonpath.spi.json.JsonProvider;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
 import org.apache.nifi.attribute.expression.language.EvaluationContext;
-import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.InvalidJsonException;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+import com.jayway.jsonpath.spi.json.JsonProvider;
 
-public class JsonPathEvaluator extends StringEvaluator {
+/**
+ * Abstract base JsonPath class with utility methods
+ *
+ * @see JsonPathEvaluator
+ * @see JsonPathDeleteEvaluator
+ */
+public abstract class JsonPathBaseEvaluator extends StringEvaluator {
 
-    private static final StringQueryResult EMPTY_RESULT = new StringQueryResult("");
-    private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
-    private static final JsonProvider JSON_PROVIDER = STRICT_PROVIDER_CONFIGURATION.jsonProvider();
+    protected static final StringQueryResult EMPTY_RESULT = new StringQueryResult("");
+    protected static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
+    protected static final JsonProvider JSON_PROVIDER = STRICT_PROVIDER_CONFIGURATION.jsonProvider();
 
-    private final Evaluator<String> subject;
-    private final Evaluator<String> jsonPathExp;
-    private final JsonPath precompiledJsonPathExp;
+    protected final Evaluator<String> subject;
+    protected final Evaluator<String> jsonPathExp;
+    protected final JsonPath precompiledJsonPathExp;
 
-    public JsonPathEvaluator(final Evaluator<String> subject, final Evaluator<String> jsonPathExp) {
+    public JsonPathBaseEvaluator(final Evaluator<String> subject, final Evaluator<String> jsonPathExp) {
         this.subject = subject;
         this.jsonPathExp = jsonPathExp;
         // if the search string is a literal, we don't need to evaluate it each
         // time; we can just
         // pre-compile it. Otherwise, it must be compiled every time.
         if (jsonPathExp instanceof StringLiteralEvaluator) {
-            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
+            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(null).getValue());
         } else {
             precompiledJsonPathExp = null;
         }
-
     }
 
-    @Override
-    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
-        final String subjectValue = subject.evaluate(evaluationContext).getValue();
+    protected DocumentContext getDocumentContext(EvaluationContext context) {
+        final String subjectValue = subject.evaluate(context).getValue();
         if (subjectValue == null || subjectValue.length() == 0) {
-            throw new  AttributeExpressionLanguageException("Subject is empty");
+            throw new AttributeExpressionLanguageException("Subject is empty");
         }
         DocumentContext documentContext = null;
         try {
@@ -73,26 +74,19 @@ public class JsonPathEvaluator extends StringEvaluator {
         } catch (InvalidJsonException e) {
             throw new AttributeExpressionLanguageException("Subject contains invalid JSON: " + subjectValue, e);
         }
+        return documentContext;
+    }
 
+    protected JsonPath getJsonPath(EvaluationContext context) {
         final JsonPath compiledJsonPath;
         if (precompiledJsonPathExp != null) {
             compiledJsonPath = precompiledJsonPathExp;
         } else {
-            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(evaluationContext).getValue());
+            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(context).getValue());
         }
-
-        Object result = null;
-        try {
-            result = documentContext.read(compiledJsonPath);
-        } catch (Exception e) {
-            // assume the path did not match anything in the document
-            return EMPTY_RESULT;
-        }
-
-        return new StringQueryResult(getResultRepresentation(result, EMPTY_RESULT.getValue()));
+        return compiledJsonPath;
     }
 
-
     @Override
     public Evaluator<?> getSubjectEvaluator() {
         return subject;
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathDeleteEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathDeleteEvaluator.java
new file mode 100644
index 0000000..9ecb77c
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathDeleteEvaluator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.attribute.expression.language.evaluation.functions;
+
+import org.apache.nifi.attribute.expression.language.EvaluationContext;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
+
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
+
+/**
+ * JsonPathDeleteEvaluator allows delete elements at the specified path
+ */
+public class JsonPathDeleteEvaluator extends JsonPathBaseEvaluator {
+
+    public JsonPathDeleteEvaluator(final Evaluator<String> subject, final Evaluator<String> jsonPathExp) {
+        super(subject, jsonPathExp);
+    }
+
+    @Override
+    public QueryResult<String> evaluate(EvaluationContext context) {
+        DocumentContext documentContext = getDocumentContext(context);
+
+        final JsonPath compiledJsonPath = getJsonPath(context);;
+
+        String result = null;
+        try {
+            result = documentContext.delete(compiledJsonPath).jsonString();
+        } catch (Exception e) {
+            // assume the path did not match anything in the document
+            return EMPTY_RESULT;
+        }
+
+        return new StringQueryResult(getResultRepresentation(result, EMPTY_RESULT.getValue()));
+    }
+
+}
+
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
index aadb324..c12a807 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/JsonPathEvaluator.java
@@ -16,70 +16,28 @@
  */
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
-import com.jayway.jsonpath.Configuration;
-import com.jayway.jsonpath.DocumentContext;
-import com.jayway.jsonpath.InvalidJsonException;
-import com.jayway.jsonpath.JsonPath;
-import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
-import com.jayway.jsonpath.spi.json.JsonProvider;
 import org.apache.nifi.attribute.expression.language.EvaluationContext;
-import org.apache.nifi.attribute.expression.language.StandardEvaluationContext;
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
 import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
 
-public class JsonPathEvaluator extends StringEvaluator {
-
-    private static final StringQueryResult EMPTY_RESULT = new StringQueryResult("");
-    private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
-    private static final JsonProvider JSON_PROVIDER = STRICT_PROVIDER_CONFIGURATION.jsonProvider();
+import com.jayway.jsonpath.DocumentContext;
+import com.jayway.jsonpath.JsonPath;
 
-    private final Evaluator<String> subject;
-    private final Evaluator<String> jsonPathExp;
-    private final JsonPath precompiledJsonPathExp;
+/**
+ * JsonPathEvaluator provides access to document at the specified JsonPath
+ */
+public class JsonPathEvaluator extends JsonPathBaseEvaluator {
 
     public JsonPathEvaluator(final Evaluator<String> subject, final Evaluator<String> jsonPathExp) {
-        this.subject = subject;
-        this.jsonPathExp = jsonPathExp;
-        // if the search string is a literal, we don't need to evaluate it each
-        // time; we can just
-        // pre-compile it. Otherwise, it must be compiled every time.
-        if (jsonPathExp instanceof StringLiteralEvaluator) {
-            precompiledJsonPathExp = compileJsonPathExpression(jsonPathExp.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue());
-        } else {
-            precompiledJsonPathExp = null;
-        }
-
+        super(subject, jsonPathExp);
     }
 
     @Override
-    public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
-        final String subjectValue = subject.evaluate(evaluationContext).getValue();
-        if (subjectValue == null || subjectValue.length() == 0) {
-            throw new  AttributeExpressionLanguageException("Subject is empty");
-        }
-        DocumentContext documentContext = null;
-        try {
-            documentContext = validateAndEstablishJsonContext(subjectValue);
-        } catch (InvalidJsonException e) {
-            throw new AttributeExpressionLanguageException("Subject contains invalid JSON: " + subjectValue, e);
-        }
+    public QueryResult<String> evaluate(EvaluationContext context) {
+        DocumentContext documentContext = getDocumentContext(context);
 
-        final JsonPath compiledJsonPath;
-        if (precompiledJsonPathExp != null) {
-            compiledJsonPath = precompiledJsonPathExp;
-        } else {
-            compiledJsonPath = compileJsonPathExpression(jsonPathExp.evaluate(evaluationContext).getValue());
-        }
+        final JsonPath compiledJsonPath = getJsonPath(context);
 
         Object result = null;
         try {
@@ -92,38 +50,5 @@ public class JsonPathEvaluator extends StringEvaluator {
         return new StringQueryResult(getResultRepresentation(result, EMPTY_RESULT.getValue()));
     }
 
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-    static DocumentContext validateAndEstablishJsonContext(final String json) {
-        final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(json);
-        return ctx;
-    }
-
-    static boolean isJsonScalar(final Object obj) {
-        return !(obj instanceof Map || obj instanceof List);
-    }
-
-    static String getResultRepresentation(final Object jsonPathResult, final String defaultValue) {
-        if (isJsonScalar(jsonPathResult)) {
-            return Objects.toString(jsonPathResult, defaultValue);
-        } else if (jsonPathResult instanceof List && ((List<?>) jsonPathResult).size() == 1) {
-            return getResultRepresentation(((List<?>) jsonPathResult).get(0), defaultValue);
-        } else {
-            return JSON_PROVIDER.toJson(jsonPathResult);
-        }
-    }
-
-    static JsonPath compileJsonPathExpression(String exp) {
-        try {
-            return JsonPath.compile(exp);
-        } catch (Exception e) {
-            throw new AttributeExpressionLanguageException("Invalid JSON Path expression: " + exp, e);
-        }
-    }
-
 }
 
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index fcd636a..4080ca0 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -324,6 +324,69 @@ public class TestQuery {
     }
 
     @Test
+    public void testJsonPathDeleteFirstNameAttribute() throws IOException {
+        final Map<String, String> attributes = new HashMap<>();
+        String addressBook = getResourceAsString("/json/address-book.json");
+        attributes.put("json", addressBook);
+
+        verifyEquals("${json:jsonPath('$.firstName')}", attributes, "John");
+
+        String addressBookAfterDelete = Query.evaluateExpressions("${json:jsonPathDelete('$.firstName')}", attributes, ParameterLookup.EMPTY);
+        attributes.clear();
+        attributes.put("json", addressBookAfterDelete);
+
+        verifyEquals("${json:jsonPath('$.lastName')}", attributes, "Smith");
+        verifyEquals("${json:jsonPath('$.age')}", attributes, "25");
+        verifyEquals("${json:jsonPath('$.address.postalCode')}", attributes, "10021-3100");
+        verifyEquals("${json:jsonPath(\"$.phoneNumbers[?(@.type=='home')].number\")}", attributes, "212 555-1234");
+        verifyEquals("${json:jsonPath('$.phoneNumbers')}", attributes,
+                "[{\"type\":\"home\",\"number\":\"212 555-1234\"},{\"type\":\"office\",\"number\":\"646 555-4567\"}]");
+
+        verifyEquals("${json:jsonPath('$.firstName')}", attributes, "");
+    }
+
+    @Test
+    public void testJsonPathDeleteMissingPath() throws IOException {
+        final Map<String, String> attributes = new HashMap<>();
+        String addressBook = getResourceAsString("/json/address-book.json");
+        attributes.put("json", addressBook);
+
+        String addressBookAfterDelete = Query.evaluateExpressions("${json:jsonPathDelete('$.missing-path')}", attributes, ParameterLookup.EMPTY);
+        attributes.clear();
+        attributes.put("json", addressBookAfterDelete);
+
+        verifyEquals("${json:jsonPath('$.firstName')}", attributes, "John");
+        verifyEquals("${json:jsonPath('$.lastName')}", attributes, "Smith");
+        verifyEquals("${json:jsonPath('$.age')}", attributes, "25");
+        verifyEquals("${json:jsonPath('$.address')}", attributes,
+                "{\"streetAddress\":\"21 2nd Street\",\"city\":\"New York\",\"state\":\"NY\",\"postalCode\":\"10021-3100\"}");
+        verifyEquals("${json:jsonPath('$.phoneNumbers')}", attributes,
+                "[{\"type\":\"home\",\"number\":\"212 555-1234\"},{\"type\":\"office\",\"number\":\"646 555-4567\"}]");
+    }
+
+    @Test
+    public void testJsonPathDeleteHomePhoneNumber() throws IOException {
+        final Map<String, String> attributes = new HashMap<>();
+        String addressBook = getResourceAsString("/json/address-book.json");
+        attributes.put("json", addressBook);
+
+        verifyEquals("${json:jsonPath(\"$.phoneNumbers[?(@.type=='home')].number\")}", attributes, "212 555-1234");
+
+        String addressBookAfterDelete = Query.evaluateExpressions("${json:jsonPathDelete(\"$.phoneNumbers[?(@.type=='home')]\")}", attributes, ParameterLookup.EMPTY);
+
+        attributes.clear();
+        attributes.put("json", addressBookAfterDelete);
+
+        verifyEquals("${json:jsonPath('$.firstName')}", attributes, "John");
+        verifyEquals("${json:jsonPath('$.lastName')}", attributes, "Smith");
+        verifyEquals("${json:jsonPath('$.age')}", attributes, "25");
+        verifyEquals("${json:jsonPath('$.address.postalCode')}", attributes, "10021-3100");
+        verifyEquals("${json:jsonPath(\"$.phoneNumbers[?(@.type=='home')].number\")}", attributes, "[]");
+        verifyEquals("${json:jsonPath('$.phoneNumbers')}", attributes,
+                "{\"type\":\"office\",\"number\":\"646 555-4567\"}");
+    }
+
+    @Test
     public void testEmbeddedExpressionsAndQuotesWithProperties() {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("x", "abc");
diff --git a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
index 9d9751a..364b5ca 100644
--- a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
@@ -1576,6 +1576,58 @@ Expressions will provide the following results:
 
 An empty subject value or a subject value with an invalid JSON document results in an exception bulletin.
 
+
+[.function]
+=== jsonPathDelete
+
+*Description*: [.description]#The `jsonPathDelete` function deletes the specified JsonPath from a Subject JSON and returns string
+form of the updated JSON.#
+
+*Subject Type*: [.subject]#String#
+
+*Arguments*:
+	 [.argName]#_jsonPath_# : [.argDesc]#the JSON path expression to delete from the Subject.#
+
+*Return Type*: [.returnType]#String#
+
+*Examples*: If the "myJson" attribute is
+
+..........
+{
+  "firstName": "John",
+  "lastName": "Smith",
+  "isAlive": true,
+  "age": 25,
+  "address": {
+    "streetAddress": "21 2nd Street",
+    "city": "New York",
+    "state": "NY",
+    "postalCode": "10021-3100"
+  },
+  "phoneNumbers": [
+    {
+      "type": "home",
+      "number": "212 555-1234"
+    },
+    {
+      "type": "office",
+      "number": "646 555-4567"
+    }
+  ],
+  "children": [],
+  "spouse": null
+}
+..........
+
+.jsonPathDelete Examples
+|===================================================================
+| Expression | Value
+| `${myJson:jsonPathDelete('$.firstName')}` | `{"lastName":"Smith","age":25,"address":{"streetAddress":"21 2nd Street","city":"New York","state":"NY","postalCode":"10021-3100"},"phoneNumbers":[{"type":"home","number":"212 555-1234"},{"type":"office","number":"646 555-4567"}]}`
+| `${myJson:jsonPathDelete('$.missing-path')}` | Returns original JSON document
+|===================================================================
+
+An empty subject value or a subject value with an invalid JSON document results in an exception bulletin.
+
 [[numbers]]
 == Mathematical Operations and Numeric Manipulation