You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by rf...@apache.org on 2019/11/14 20:38:11 UTC

[nifi] branch master updated: NIFI-6868: Ensure that when a property value is obtained by calling ProcessContext.getPropertyValue(...).evaluateAttributeExpressions(...).getValue() that we still retain the appropriate value even if the property references a sensitive parameter * Added additional unit tests and updated ParameterEvaluator to make sure that it follows the same logic as the ParameterExpression in that sensitive parameters cannot be access from within EL

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

rfellows 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 ea31c1f  NIFI-6868: Ensure that when a property value is obtained by calling ProcessContext.getPropertyValue(...).evaluateAttributeExpressions(...).getValue() that we still retain the appropriate value even if the property references a sensitive parameter * Added additional unit tests and updated ParameterEvaluator to make sure that it follows the same logic as the ParameterExpression in that sensitive parameters cannot be access from within EL
ea31c1f is described below

commit ea31c1fcbb270031339f75b1d6ca4a157fdb7594
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu Nov 14 13:49:20 2019 -0500

    NIFI-6868: Ensure that when a property value is obtained by calling ProcessContext.getPropertyValue(...).evaluateAttributeExpressions(...).getValue() that we still retain the appropriate value even if the property references a sensitive parameter
    * Added additional unit tests and updated ParameterEvaluator to make sure that it follows the same logic as the ParameterExpression in that sensitive parameters cannot be access from within EL
    
    This closes #3889
    Signed-off-by: Rob Fellows <ro...@gmail.com>
---
 .../expression/language/ParameterExpression.java   | 14 ++++-
 .../nifi/attribute/expression/language/Query.java  | 20 ++++---
 .../evaluation/selection/ParameterEvaluator.java   |  2 +-
 .../language/TestStandardPreparedQuery.java        | 32 ++++++++++
 .../nifi/integration/parameters/ParametersIT.java  | 69 ++++++++++++++++++++++
 5 files changed, 126 insertions(+), 11 deletions(-)

diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
index 914a4d5..fe8595f 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ParameterExpression.java
@@ -21,14 +21,24 @@ import org.apache.nifi.parameter.Parameter;
 
 public class ParameterExpression implements Expression {
     private final String parameterName;
+    private final boolean allowSensitiveParameterReference;
 
-    public ParameterExpression(final String parameterName) {
+    public ParameterExpression(final String parameterName, final boolean allowSensitiveParameterReference) {
         this.parameterName = parameterName;
+        this.allowSensitiveParameterReference = allowSensitiveParameterReference;
     }
 
     @Override
     public String evaluate(final EvaluationContext evaluationContext, final AttributeValueDecorator decorator) {
         final Parameter parameter = evaluationContext.getParameter(parameterName);
-        return parameter == null || parameter.getDescriptor().isSensitive() ? null : parameter.getValue();
+        if (parameter == null) {
+            return null;
+        }
+
+        if (!allowSensitiveParameterReference && parameter.getDescriptor().isSensitive()) {
+            return null;
+        }
+
+        return parameter.getValue();
     }
 }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
index e0f4cf0..5cbdf78 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
@@ -332,19 +332,19 @@ public class Query {
 
                 if (range.getStart() > lastIndex) {
                     String substring = unescapeLeadingDollarSigns(query.substring(lastIndex, range.getStart()));
-                    addLiteralsAndParameters(parameterParser, substring, expressions);
+                    addLiteralsAndParameters(parameterParser, substring, expressions, true);
                 }
 
-                addLiteralsAndParameters(parameterParser, treeText, expressions);
+                addLiteralsAndParameters(parameterParser, treeText, expressions, true);
             }
 
             if (escapedRanges.isEmpty()) {
-                addLiteralsAndParameters(parameterParser, query, expressions);
+                addLiteralsAndParameters(parameterParser, query, expressions, true);
             } else {
                 final Range lastRange = escapedRanges.get(escapedRanges.size() - 1);
                 if (lastRange.getEnd() + 1 < query.length()) {
                     final String treeText = unescapeLeadingDollarSigns(query.substring(lastRange.getEnd() + 1));
-                    addLiteralsAndParameters(parameterParser, treeText, expressions);
+                    addLiteralsAndParameters(parameterParser, treeText, expressions, true);
                 }
             }
 
@@ -375,7 +375,11 @@ public class Query {
                         substring = unescapeTrailingDollarSigns(substring, false);
                     }
 
-                    addLiteralsAndParameters(parameterParser, substring, expressions);
+                    // Do not allow sensitive parameters to be referenced because this is within an actual Expression.
+                    // For example, ${#{sensitiveParam}} is not allowed. However, we do support referencing sensitive parameters
+                    // for the use case of simply #{sensitiveParam} outside of an Expression. In such a case, the PreparedQuery will
+                    // still be used to evaluate this, since all Property Values are evaluated through PreparedQueries.
+                    addLiteralsAndParameters(parameterParser, substring, expressions, false);
                 }
 
                 expressions.add(compiledExpression);
@@ -386,7 +390,7 @@ public class Query {
             final Range lastRange = ranges.get(ranges.size() - 1);
             if (lastRange.getEnd() + 1 < query.length()) {
                 final String treeText = unescapeLeadingDollarSigns(query.substring(lastRange.getEnd() + 1));
-                addLiteralsAndParameters(parameterParser, treeText, expressions);
+                addLiteralsAndParameters(parameterParser, treeText, expressions, false);
             }
 
             return new StandardPreparedQuery(expressions);
@@ -395,7 +399,7 @@ public class Query {
         }
     }
 
-    private static void addLiteralsAndParameters(final ParameterParser parser, final String input, final List<Expression> expressions) {
+    private static void addLiteralsAndParameters(final ParameterParser parser, final String input, final List<Expression> expressions, final boolean allowSensitiveParameterReference) {
         final ParameterTokenList references = parser.parseTokens(input);
         int index = 0;
 
@@ -416,7 +420,7 @@ public class Query {
 
             if (token.isParameterReference()) {
                 final ParameterReference parameterReference = (ParameterReference) token;
-                expressions.add(new ParameterExpression(parameterReference.getParameterName()));
+                expressions.add(new ParameterExpression(parameterReference.getParameterName(), allowSensitiveParameterReference));
             } else {
                 expressions.add(new StringLiteralExpression(token.getValue(ParameterLookup.EMPTY)));
             }
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/ParameterEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/ParameterEvaluator.java
index a5f2585..1c77d2b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/ParameterEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/ParameterEvaluator.java
@@ -33,7 +33,7 @@ public class ParameterEvaluator extends StringEvaluator {
     @Override
     public QueryResult<String> evaluate(final EvaluationContext evaluationContext) {
         final Parameter parameter = evaluationContext.getParameter(parameterName);
-        return new StringQueryResult(parameter == null ? null : parameter.getValue());
+        return new StringQueryResult( (parameter == null || parameter.getDescriptor().isSensitive()) ? null : parameter.getValue());
     }
 
     @Override
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 d6767ba..09c50d9 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
@@ -16,13 +16,17 @@
  */
 package org.apache.nifi.attribute.expression.language;
 
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
 import org.apache.nifi.parameter.ParameterLookup;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
@@ -237,6 +241,34 @@ public class TestStandardPreparedQuery {
     }
 
     @Test
+    public void testSensitiveParameter() {
+        final Map<String, Parameter> parameters = new HashMap<>();
+        parameters.put("param", new Parameter(new ParameterDescriptor.Builder().name("param").build(), "value"));
+        parameters.put("sensi", new Parameter(new ParameterDescriptor.Builder().name("sensi").sensitive(true).build(), "secret"));
+
+        final ParameterLookup parameterLookup = new ParameterLookup() {
+            @Override
+            public Optional<Parameter> getParameter(final String parameterName) {
+                return Optional.ofNullable(parameters.get(parameterName));
+            }
+
+            @Override
+            public boolean isEmpty() {
+                return parameters.isEmpty();
+            }
+        };
+
+        final String value = Query.prepare("#{param}").evaluateExpressions(new StandardEvaluationContext(Collections.emptyMap(), Collections.emptyMap(), parameterLookup), null);
+        assertEquals("value", value);
+
+        final String secret = Query.prepare("#{sensi}").evaluateExpressions(new StandardEvaluationContext(Collections.emptyMap(), Collections.emptyMap(), parameterLookup), null);
+        assertEquals("secret", secret);
+
+        final String invalid = Query.prepare("${#{sensi}}").evaluateExpressions(new StandardEvaluationContext(Collections.emptyMap(), Collections.emptyMap(), parameterLookup), null);
+        assertEquals("", invalid);
+    }
+
+    @Test
     public void testVariableImpacted() {
         final Set<String> attr = new HashSet<>();
         attr.add("attr");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/parameters/ParametersIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/parameters/ParametersIT.java
index 33d7639..6e47264 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/parameters/ParametersIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/parameters/ParametersIT.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.integration.parameters;
 
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.queue.FlowFileQueue;
@@ -25,6 +27,7 @@ import org.apache.nifi.integration.processors.GenerateProcessor;
 import org.apache.nifi.integration.processors.UpdateAttributeCreateOwnProperty;
 import org.apache.nifi.integration.processors.UpdateAttributeNoEL;
 import org.apache.nifi.integration.processors.UpdateAttributeWithEL;
+import org.apache.nifi.integration.processors.UsernamePasswordProcessor;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterDescriptor;
@@ -32,7 +35,10 @@ import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.parameter.ParameterReferenceManager;
 import org.apache.nifi.parameter.StandardParameterContext;
 import org.apache.nifi.parameter.StandardParameterReferenceManager;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.StandardProcessContext;
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -364,8 +370,71 @@ public class ParametersIT extends FrameworkIntegrationTest {
         updateAttribute.setProperties(properties);
         referencedParameters = updateAttribute.getReferencedParameterNames();
         assertEquals(allParamNames, referencedParameters);
+    }
+
+    @Test
+    public void testSensitivePropertyReferenceParameterSupportsEL() {
+        final ProcessorNode usernamePassword = createProcessorNode(UsernamePasswordProcessor.class);
+
+        final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
+        final ParameterContext parameterContext = new StandardParameterContext(UUID.randomUUID().toString(), "param-context", referenceManager, null);
+        parameterContext.setParameters(Collections.singletonMap("pass", new Parameter(new ParameterDescriptor.Builder().name("pass").sensitive(true).build(), "secret")));
+
+        getRootGroup().setParameterContext(parameterContext);
+
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("password", "#{pass}");
+        usernamePassword.setProperties(properties);
+
+        final ProcessContext processContext = new StandardProcessContext(usernamePassword, getFlowController().getControllerServiceProvider(), getFlowController().getEncryptor(),
+            getFlowController().getStateManagerProvider().getStateManager(usernamePassword.getIdentifier()), () -> false);
+        final PropertyDescriptor descriptor = usernamePassword.getPropertyDescriptor("password");
+        final PropertyValue propertyValue = processContext.getProperty(descriptor);
+        final PropertyValue evaluatedPropertyValue = propertyValue.evaluateAttributeExpressions();
+        final String evaluatedPassword = evaluatedPropertyValue.getValue();
+        assertEquals("secret", evaluatedPassword);
+    }
+
+    @Test
+    public void testSensitivePropertyNotAccessibleFromWithinEL() {
+        final ProcessorNode usernamePassword = createProcessorNode(UsernamePasswordProcessor.class);
+
+        final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
+        final ParameterContext parameterContext = new StandardParameterContext(UUID.randomUUID().toString(), "param-context", referenceManager, null);
+        parameterContext.setParameters(Collections.singletonMap("pass", new Parameter(new ParameterDescriptor.Builder().name("pass").sensitive(true).build(), "secret")));
+
+        getRootGroup().setParameterContext(parameterContext);
 
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("username", "${#{pass}}");
+
+        try {
+            usernamePassword.setProperties(properties);
+            Assert.fail("Was able to set properties when referencing sensitive parameter from within EL");
+        } catch (final IllegalArgumentException iae) {
+            // Expected. Since the parameter is sensitive, it may referenced by a sensitive property
+        }
     }
 
+    @Test
+    public void testSensitivePropertyCannotBeSetToReferenceParamFromEL() {
+        final ProcessorNode usernamePassword = createProcessorNode(UsernamePasswordProcessor.class);
+
+        final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(getFlowController().getFlowManager());
+        final ParameterContext parameterContext = new StandardParameterContext(UUID.randomUUID().toString(), "param-context", referenceManager, null);
+        parameterContext.setParameters(Collections.singletonMap("pass", new Parameter(new ParameterDescriptor.Builder().name("pass").sensitive(true).build(), "secret")));
+
+        getRootGroup().setParameterContext(parameterContext);
+
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("password", "${#{pass}}");
+
+        try {
+            usernamePassword.setProperties(properties);
+            Assert.fail("Was able to set properties when referencing sensitive parameter from within EL");
+        } catch (final IllegalArgumentException iae) {
+            // Expected. Since the property is sensitive, it may reference a parameter only if that is the only value.
+        }
+    }
 
 }