You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2022/10/21 18:19:52 UTC

[nifi] branch main updated: NIFI-10674 Block evaluateELString from reading Parameters (#6562)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 3e9b7e27a5 NIFI-10674 Block evaluateELString from reading Parameters (#6562)
3e9b7e27a5 is described below

commit 3e9b7e27a54a333201fe53a9fd89a7cbbd318569
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Fri Oct 21 13:19:41 2022 -0500

    NIFI-10674 Block evaluateELString from reading Parameters (#6562)
    
    * NIFI-10674 Blocked access to Parameters from evaluateELString()
---
 .../ParametersDisabledEvaluationContext.java       |  65 ++++++++++++
 .../functions/EvaluateELStringEvaluator.java       |   5 +-
 .../attribute/expression/language/TestQuery.java   | 115 +++++++++++----------
 .../language/TestStandardPreparedQuery.java        |  48 ++++++---
 4 files changed, 159 insertions(+), 74 deletions(-)

diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParametersDisabledEvaluationContext.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParametersDisabledEvaluationContext.java
new file mode 100644
index 0000000000..105a9fdbbe
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParametersDisabledEvaluationContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
+import org.apache.nifi.parameter.Parameter;
+
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Delegating implementation of Evaluation Context that blocks access to Parameter Values
+ */
+public class ParametersDisabledEvaluationContext implements EvaluationContext {
+    private final EvaluationContext evaluationContext;
+
+    public ParametersDisabledEvaluationContext(final EvaluationContext evaluationContext) {
+        this.evaluationContext = Objects.requireNonNull(evaluationContext, "Evaluation Context required");
+    }
+
+    @Override
+    public String getExpressionValue(final String name) {
+        return evaluationContext.getExpressionValue(name);
+    }
+
+    @Override
+    public Set<String> getExpressionKeys() {
+        return evaluationContext.getExpressionKeys();
+    }
+
+    @Override
+    public String getState(final String key) {
+        return evaluationContext.getState(key);
+    }
+
+    @Override
+    public EvaluatorState getEvaluatorState() {
+        return evaluationContext.getEvaluatorState();
+    }
+
+    /**
+     * Get Parameter returns null in all cases to block access to Parameter Values
+     *
+     * @param parameterName Parameter Name
+     * @return Null Parameter
+     */
+    @Override
+    public Parameter getParameter(final String parameterName) {
+        return null;
+    }
+}
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EvaluateELStringEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EvaluateELStringEvaluator.java
index eaa3089ff9..dd544423fb 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EvaluateELStringEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EvaluateELStringEvaluator.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.attribute.expression.language.evaluation.functions;
 
 import org.apache.nifi.attribute.expression.language.EvaluationContext;
+import org.apache.nifi.attribute.expression.language.ParametersDisabledEvaluationContext;
 import org.apache.nifi.attribute.expression.language.Query;
-import org.apache.nifi.attribute.expression.language.StandardPreparedQuery;
 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;
@@ -35,7 +35,8 @@ public class EvaluateELStringEvaluator extends StringEvaluator {
     @Override
     public QueryResult<String> evaluate(EvaluationContext evaluationContext) {
         final String subjectValue = subject.evaluate(evaluationContext).getValue();
-        final String evaluated = ((StandardPreparedQuery) Query.prepare(subjectValue)).evaluateExpressions(evaluationContext, null);
+        final ParametersDisabledEvaluationContext parametersDisabledEvaluationContext = new ParametersDisabledEvaluationContext(evaluationContext);
+        final String evaluated = Query.prepare(subjectValue).evaluateExpressions(parametersDisabledEvaluationContext, null);
         return new StringQueryResult(evaluated);
     }
 
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 87f7e782e5..7656548089 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
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
-import org.antlr.runtime.tree.Tree;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
@@ -31,7 +30,6 @@ import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.registry.VariableRegistry;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
 
 import java.io.BufferedInputStream;
 import java.io.IOException;
@@ -64,6 +62,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestQuery {
 
@@ -110,8 +111,6 @@ public class TestQuery {
         assertValid("${attr:padRight(10, '#')}");
         assertValid("${attr:padLeft(10)}");
         assertValid("${attr:padRight(10)}");
-        // left here because it's convenient for looking at the output
-        //System.out.println(Query.compile("").evaluate(null));
     }
 
 
@@ -184,12 +183,47 @@ public class TestQuery {
         verifyEquals(query3, attrs, "male");
     }
 
+    @Test
+    public void testEvaluateELStringNonSensitiveParameters() {
+        final String value = "value";
+        final String query = "${variable:evaluateELString()}";
+
+        final Map<String, String> variables = Collections.singletonMap("variable", "#{parameter}");
+        final Map<String, String> parameters = Collections.singletonMap("parameter", value);
+
+        final MapParameterLookup parameterLookup = new MapParameterLookup(parameters);
+        final EvaluationContext evaluationContext = new StandardEvaluationContext(variables, Collections.emptyMap(), parameterLookup);
+
+        final String evaluated = Query.prepare(query).evaluateExpressions(evaluationContext, null);
+
+        assertEquals(StringUtils.EMPTY, evaluated);
+    }
+
+    @Test
+    public void testEvaluateELStringSensitiveParameters() {
+        final String parameterName = "parameter";
+        final String parameterRef = String.format("#{%s}", parameterName);
+        final String value = "value";
+        final String query = "${variable:evaluateELString()}";
+
+        final Map<String, String> variables = Collections.singletonMap("variable", parameterRef);
+
+        final ParameterLookup parameterLookup = mock(ParameterLookup.class);
+        final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder().name(parameterName).sensitive(true).build();
+        final Parameter parameter = new Parameter(parameterDescriptor, value);
+        when(parameterLookup.getParameter(eq(parameterName))).thenReturn(Optional.of(parameter));
+        when(parameterLookup.isEmpty()).thenReturn(false);
+
+        final EvaluationContext evaluationContext = new StandardEvaluationContext(variables, Collections.emptyMap(), parameterLookup);
+
+        final String evaluated = Query.prepare(query).evaluateExpressions(evaluationContext, null);
+
+        assertEquals(StringUtils.EMPTY, evaluated);
+    }
+
     @Test
     public void testCompileEmbedded() {
         final String expression = "${x:equals( ${y} )}";
-        final Query query = Query.compile(expression);
-        final Tree tree = query.getTree();
-        System.out.println(printTree(tree));
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("x", "x");
@@ -200,29 +234,6 @@ public class TestQuery {
         Query.validateExpression(expression, false);
     }
 
-    private String printTree(final Tree tree) {
-        final StringBuilder sb = new StringBuilder();
-        printTree(tree, 0, sb);
-
-        return sb.toString();
-    }
-
-    private void printTree(final Tree tree, final int spaces, final StringBuilder sb) {
-        for (int i = 0; i < spaces; i++) {
-            sb.append(" ");
-        }
-
-        if (tree.getText().trim().isEmpty()) {
-            sb.append(tree.toString()).append("\n");
-        } else {
-            sb.append(tree.getText()).append("\n");
-        }
-
-        for (int i = 0; i < tree.getChildCount(); i++) {
-            printTree(tree.getChild(i), spaces + 2, sb);
-        }
-    }
-
     @Test
     public void testEscape() {
         final Map<String, String> attributes = new HashMap<>();
@@ -433,7 +444,6 @@ public class TestQuery {
         final Map<String, String> parameters = new HashMap<>();
         parameters.put("test param", "unit");
 
-        final Query query = Query.compile("${'#{test param}'}");
         verifyEquals("${#{'test param'}}", attributes, stateValues, parameters,"unit");
         verifyEquals("${#{'test param'}:append(' - '):append(#{'test param'})}", attributes, stateValues, parameters,"unit - unit");
 
@@ -589,7 +599,8 @@ public class TestQuery {
        verifyEquals("${json:jsonPath('$.missing-path')}", attributes, "");
     }
 
-    public void testJsonPathAddNicknameJimmyAtNonArray() throws IOException {
+    @Test
+    public void testJsonPathAddNicknameJimmyAtNonArray() {
         assertThrows(IllegalArgumentException.class, () -> verifyJsonPathExpressions(
                 ADDRESS_BOOK_JSON_PATH_EMPTY,
                 "",
@@ -629,7 +640,7 @@ public class TestQuery {
 
     @Test
     public void testJsonPathPutOverwriteFirstNameToJimmy() throws IOException {
-        Map<String,String> attributes = verifyJsonPathExpressions(
+        verifyJsonPathExpressions(
                 ADDRESS_BOOK_JSON_PATH_FIRST_NAME,
                 "John",
                 "${json:jsonPathPut('$','firstName','Jimmy')}",
@@ -720,12 +731,12 @@ public class TestQuery {
 
     @SuppressWarnings("unchecked")
     private String evaluateQueryForEscape(final String queryString, final Map<String, String> attributes) {
-        final FlowFile mockFlowFile = Mockito.mock(FlowFile.class);
-        Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes);
-        Mockito.when(mockFlowFile.getId()).thenReturn(1L);
-        Mockito.when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis());
-        Mockito.when(mockFlowFile.getSize()).thenReturn(1L);
-        Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
+        final FlowFile mockFlowFile = mock(FlowFile.class);
+        when(mockFlowFile.getAttributes()).thenReturn(attributes);
+        when(mockFlowFile.getId()).thenReturn(1L);
+        when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis());
+        when(mockFlowFile.getSize()).thenReturn(1L);
+        when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
 
         final ValueLookup lookup = new ValueLookup(VariableRegistry.EMPTY_REGISTRY, mockFlowFile);
         return Query.evaluateExpressions(queryString, lookup, ParameterLookup.EMPTY);
@@ -909,7 +920,6 @@ public class TestQuery {
         verifyEquals(query, attributes, "say \"hi\"");
 
         query = "${xx:replace( '\\'', '\"')}";
-        System.out.println(query);
         verifyEquals(query, attributes, "say \"hi\"");
     }
 
@@ -919,7 +929,6 @@ public class TestQuery {
         attributes.put("xx", "say 'hi'");
 
         final String query = "${xx:replace( \"'hi'\", '\\\"hello\\\"' )}";
-        System.out.println(query);
         verifyEquals(query, attributes, "say \"hello\"");
     }
 
@@ -1667,9 +1676,9 @@ public class TestQuery {
         verifyEmpty("${literal('0x1.1'):toDecimal()}", attributes);
 
         // Special cases
-        verifyEquals("${literal('" + Double.toString(POSITIVE_INFINITY) + "'):toDecimal():plus(1):plus(2)}", attributes, POSITIVE_INFINITY);
-        verifyEquals("${literal('" + Double.toString(NEGATIVE_INFINITY) + "'):toDecimal():plus(1):plus(2)}", attributes, NEGATIVE_INFINITY);
-        verifyEquals("${literal('" + Double.toString(NaN) + "'):toDecimal():plus(1):plus(2)}", attributes, NaN);
+        verifyEquals("${literal('" + POSITIVE_INFINITY + "'):toDecimal():plus(1):plus(2)}", attributes, POSITIVE_INFINITY);
+        verifyEquals("${literal('" + NEGATIVE_INFINITY + "'):toDecimal():plus(1):plus(2)}", attributes, NEGATIVE_INFINITY);
+        verifyEquals("${literal('" + NaN + "'):toDecimal():plus(1):plus(2)}", attributes, NaN);
     }
 
     @Test
@@ -1680,8 +1689,6 @@ public class TestQuery {
         attributes.put("hello", "world!");
         attributes.put("123.cba", "hell.o");
 
-        System.out.println(printTree(Query.compile("${allMatchingAttributes('(abc|xyz)'):matches('\\\\d+')}").getTree()));
-
         verifyEquals("${'123.cba':matches('hell\\.o')}", attributes, true);
         verifyEquals("${allMatchingAttributes('123\\.cba'):equals('hell.o')}", attributes, true);
         verifyEquals("${allMatchingAttributes('(abc|xyz)'):matches('\\d+')}", attributes, true);
@@ -1853,7 +1860,7 @@ public class TestQuery {
 
     @Test
     public void testNot() {
-        verifyEquals("${ab:notNull():not()}", new HashMap<String, String>(), true);
+        verifyEquals("${ab:notNull():not()}", new HashMap<>(), true);
     }
 
     @Test
@@ -1924,7 +1931,6 @@ public class TestQuery {
                 + "     )"
                 + "}";
 
-        System.out.println(query);
         verifyEquals(query, attributes, true);
     }
 
@@ -2037,27 +2043,26 @@ public class TestQuery {
 
     @Test
     public void testLiteralFunction() {
-        final Map<String, String> attrs = Collections.<String, String>emptyMap();
+        final Map<String, String> attrs = Collections.emptyMap();
         verifyEquals("${literal(2):gt(1)}", attrs, true);
         verifyEquals("${literal('hello'):substring(0, 1):equals('h')}", attrs, true);
     }
 
     @Test
     public void testRandomFunction() {
-        final Map<String, String> attrs = Collections.<String, String>emptyMap();
-        final Long negOne = Long.valueOf(-1L);
+        final Map<String, String> attrs = Collections.emptyMap();
+        final Long negOne = -1L;
         final HashSet<Long> results = new HashSet<>(100);
         for (int i = 0; i < results.size(); i++) {
             long result = (Long) getResult("${random()}", attrs).getValue();
             assertThat("random", result, greaterThan(negOne));
-            assertEquals(true, results.add(result), "duplicate random");
+            assertTrue(results.add(result), "duplicate random");
         }
     }
 
     QueryResult<?> getResult(String expr, Map<String, String> attrs) {
         final Query query = Query.compile(expr);
-        final QueryResult<?> result = query.evaluate(new StandardEvaluationContext(attrs));
-        return result;
+        return query.evaluate(new StandardEvaluationContext(attrs));
     }
 
     @Test
@@ -2407,7 +2412,7 @@ public class TestQuery {
 
     private void verifyEmpty(final String expression, final Map<String, String> attributes) {
         Query.validateExpression(expression, false);
-        assertEquals(String.valueOf(""), Query.evaluateExpressions(expression, attributes, null));
+        assertEquals("", Query.evaluateExpressions(expression, attributes, null));
     }
 
     private String getResourceAsString(String resourceName) throws IOException {
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
index 937f6f6c86..39f6b65be0 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -24,15 +24,18 @@ import org.junit.jupiter.api.Test;
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestStandardPreparedQuery {
 
@@ -274,19 +277,31 @@ public class TestStandardPreparedQuery {
     }
 
     @Test
-    public void testVariableImpacted() {
-        final Set<String> attr = new HashSet<>();
-        attr.add("attr");
+    public void testEvaluateExpressionLanguageVariableValueSensitiveParameterReference() {
+        final String parameterName = "protected";
+        final String parameterValue = "secret";
+
+        final ParameterLookup parameterLookup = mock(ParameterLookup.class);
+        final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder().name(parameterName).sensitive(true).build();
+        final Parameter parameter = new Parameter(parameterDescriptor, parameterValue);
+        when(parameterLookup.getParameter(eq(parameterName))).thenReturn(Optional.of(parameter));
+        when(parameterLookup.isEmpty()).thenReturn(false);
 
-        final Set<String> attr2 = new HashSet<>();
-        attr2.add("attr");
-        attr2.add("attr2");
+        final Map<String, String> variables = new LinkedHashMap<>();
+        final String variableName = "variable";
+        final String variableValue = String.format("#{%s}", parameterName);
+        variables.put(variableName, variableValue);
 
-        final Set<String> abc = new HashSet<>();
-        abc.add("a");
-        abc.add("b");
-        abc.add("c");
+        final StandardEvaluationContext context = new StandardEvaluationContext(variables, Collections.emptyMap(), parameterLookup);
 
+        final String queryExpression = String.format("${%s:evaluateELString()}", variableName);
+        final String value = Query.prepare(queryExpression).evaluateExpressions(context, null);
+
+        assertNotEquals(parameterValue, value);
+    }
+
+    @Test
+    public void testVariableImpacted() {
         assertTrue(Query.prepare("${attr}").getVariableImpact().isImpacted("attr"));
         assertFalse(Query.prepare("${attr}").getVariableImpact().isImpacted("attr2"));
         assertTrue(Query.prepare("${attr:trim():toUpper():equals('abc')}").getVariableImpact().isImpacted("attr"));
@@ -328,8 +343,7 @@ public class TestStandardPreparedQuery {
     }
 
     private String evaluate(final String query, final Map<String, String> attrs) {
-        final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(new StandardEvaluationContext(attrs), null);
-        return evaluated;
+        return Query.prepare(query).evaluateExpressions(new StandardEvaluationContext(attrs), null);
     }
 
 }