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/05/23 21:21:02 UTC
[nifi] branch main updated: NIFI-9958 Add Framework Support for Sensitive Dynamic Properties (#6057)
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 99be62be67 NIFI-9958 Add Framework Support for Sensitive Dynamic Properties (#6057)
99be62be67 is described below
commit 99be62be67a3eb850839ff187950cca19eed0c82
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Mon May 23 16:20:54 2022 -0500
NIFI-9958 Add Framework Support for Sensitive Dynamic Properties (#6057)
* NIFI-9958 Implemented Sensitive Dynamic Properties
- Added SupportsSensitiveDynamicProperties annotation for components
- Added optional sensitive query parameter to Property Descriptor REST Resource methods
- Added system tests for components supporting sensitive dynamic properties
- Updated REST Resources to support Sensitive Dynamic Property Names
- Updated Documentation Writer to indicate component support for Sensitive Dynamic Properties
- Updated InvokeHTTP to support Sensitive Dynamic Properties
- Updated Auditor components to handle masking Sensitive Dynamic Properties
* Refactored Property Descriptor REST method handling
- Corrected AbstractDocumentationWriter evaluation of support for sensitive dynamic properties
- Refactored Controller Service Dynamic Properties system tests to new class
* Updated AbstractComponentNode.getProperties() to get canonical descriptor
---
.../SupportsSensitiveDynamicProperties.java | 36 ++++++++
.../documentation/AbstractDocumentationWriter.java | 4 +
.../documentation/xml/XmlDocumentationWriter.java | 9 ++
.../nifi/web/api/dto/ControllerServiceDTO.java | 30 ++++++
.../nifi/web/api/dto/ProcessorConfigDTO.java | 15 +++
.../org/apache/nifi/web/api/dto/ProcessorDTO.java | 15 +++
.../apache/nifi/web/api/dto/ReportingTaskDTO.java | 31 +++++++
.../reporting/AbstractReportingContext.java | 11 ++-
.../service/StandardControllerServiceNode.java | 4 +-
.../StandardVersionedComponentSynchronizer.java | 88 ++++++++++++------
.../nifi/processor/StandardProcessContext.java | 5 +-
...StandardVersionedComponentSynchronizerTest.java | 8 +-
.../nifi/controller/AbstractComponentNode.java | 70 ++++++++++----
.../org/apache/nifi/controller/ComponentNode.java | 31 +++++--
.../nifi/controller/TestAbstractComponentNode.java | 79 +++++++++++++++-
.../nifi/controller/StandardFlowSnippet.java | 6 +-
.../org/apache/nifi/controller/TemplateUtils.java | 1 +
.../nifi/controller/XmlFlowSynchronizer.java | 6 +-
.../reporting/StandardReportingContext.java | 6 +-
.../reporting/StandardReportingTaskNode.java | 4 +-
.../serialization/FlowFromDOMFactory.java | 24 ++++-
.../serialization/VersionedFlowSynchronizer.java | 49 ++++++++--
.../service/ControllerServiceLoader.java | 3 +-
.../nifi/audit/ControllerServiceAuditor.java | 13 ++-
.../java/org/apache/nifi/audit/NiFiAuditor.java | 2 +
.../org/apache/nifi/audit/ProcessorAuditor.java | 19 +++-
.../apache/nifi/audit/ReportingTaskAuditor.java | 13 ++-
.../apache/nifi/web/StandardNiFiServiceFacade.java | 4 +-
.../nifi/web/api/ControllerServiceResource.java | 13 ++-
.../org/apache/nifi/web/api/ProcessorResource.java | 13 ++-
.../apache/nifi/web/api/ReportingTaskResource.java | 13 ++-
.../org/apache/nifi/web/api/dto/DtoFactory.java | 20 +++-
.../web/dao/impl/StandardControllerServiceDAO.java | 3 +-
.../nifi/web/dao/impl/StandardProcessorDAO.java | 4 +-
.../web/dao/impl/StandardReportingTaskDAO.java | 4 +-
.../nifi/web/dao/impl/StandardSnippetDAO.java | 32 +++++--
.../nifi/processors/standard/InvokeHTTP.java | 2 +
.../nifi/registry/flow/diff/DifferenceType.java | 5 +
.../registry/flow/diff/StandardFlowComparator.java | 13 ++-
.../flow/diff/StaticDifferenceDescriptor.java | 3 +
.../reporting/StatelessReportingContext.java | 6 +-
.../reporting/StatelessReportingTaskNode.java | 2 +-
.../system/SensitiveDynamicPropertiesService.java | 31 +++++++
.../SensitiveDynamicPropertiesProcessor.java | 74 +++++++++++++++
.../SensitiveDynamicPropertiesReportingTask.java | 36 ++++++++
.../org.apache.nifi.controller.ControllerService | 1 +
.../services/org.apache.nifi.processor.Processor | 1 +
.../org.apache.nifi.reporting.ReportingTask | 1 +
.../apache/nifi/tests/system/NiFiClientUtil.java | 60 ++++++++----
.../ControllerServiceApiValidationIT.java | 1 +
.../ControllerServiceDynamicPropertiesIT.java | 100 ++++++++++++++++++++
.../processor/ProcessorDynamicPropertiesIT.java | 102 +++++++++++++++++++++
.../system/reportingtask/ReportingTaskIT.java | 71 ++++++++++++++
.../impl/client/nifi/ControllerServicesClient.java | 2 +
.../cli/impl/client/nifi/ProcessorClient.java | 3 +
.../cli/impl/client/nifi/ReportingTasksClient.java | 2 +
.../nifi/impl/JerseyControllerServicesClient.java | 18 ++++
.../client/nifi/impl/JerseyProcessorClient.java | 18 ++++
.../nifi/impl/JerseyReportingTasksClient.java | 17 ++++
59 files changed, 1110 insertions(+), 147 deletions(-)
diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsSensitiveDynamicProperties.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsSensitiveDynamicProperties.java
new file mode 100644
index 0000000000..992764172d
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsSensitiveDynamicProperties.java
@@ -0,0 +1,36 @@
+/*
+ * 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.annotation.behavior;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Component annotation indicating support for dynamic properties that can be designated as sensitive for the purpose of
+ * persistence and framework processing
+ */
+@Documented
+@Target({ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+@Inherited
+public @interface SupportsSensitiveDynamicProperties {
+
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java b/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
index fe1ac2b0c9..3e88dc6d9b 100644
--- a/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
@@ -28,6 +28,7 @@ import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable;
@@ -128,6 +129,7 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
writeTags(getTags(component));
writeProperties(component.getPropertyDescriptors(), propertyServices);
writeDynamicProperties(getDynamicProperties(component));
+ writeSupportsSensitiveDynamicProperties(component.getClass().getAnnotation(SupportsSensitiveDynamicProperties.class));
if (component instanceof Processor) {
final Processor processor = (Processor) component;
@@ -302,6 +304,8 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
protected abstract void writeSupportsBatching(SupportsBatching supportsBatching) throws IOException;
+ protected abstract void writeSupportsSensitiveDynamicProperties(SupportsSensitiveDynamicProperties supportsSensitiveDynamicProperties) throws IOException;
+
protected abstract void writeEventDriven(EventDriven eventDriven) throws IOException;
protected abstract void writePrimaryNodeOnly(PrimaryNodeOnly primaryNodeOnly) throws IOException;
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java b/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
index 8b87237f2f..07c12560cb 100644
--- a/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
@@ -27,6 +27,7 @@ import org.apache.nifi.annotation.behavior.Restriction;
import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable;
@@ -462,6 +463,14 @@ public class XmlDocumentationWriter extends AbstractDocumentationWriter {
writeBooleanElement("supportsBatching", true);
}
+ @Override
+ protected void writeSupportsSensitiveDynamicProperties(final SupportsSensitiveDynamicProperties supportsSensitiveDynamicProperties) throws IOException {
+ if (supportsSensitiveDynamicProperties == null) {
+ return;
+ }
+ writeBooleanElement("supportsSensitiveDynamicProperties", true);
+ }
+
@Override
protected void writeEventDriven(EventDriven eventDriven) throws IOException {
if (eventDriven == null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java
index cb3824f29a..1c92e233a1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java
@@ -45,9 +45,11 @@ public class ControllerServiceDTO extends ComponentDTO {
private Boolean deprecated;
private Boolean isExtensionMissing;
private Boolean multipleVersionsAvailable;
+ private Boolean supportsSensitiveDynamicProperties;
private Map<String, String> properties;
private Map<String, PropertyDescriptorDTO> descriptors;
+ private Set<String> sensitiveDynamicPropertyNames;
private String customUiUrl;
private String annotationData;
@@ -201,6 +203,20 @@ public class ControllerServiceDTO extends ComponentDTO {
this.multipleVersionsAvailable = multipleVersionsAvailable;
}
+ /**
+ * @return whether this controller service supports sensitive dynamic properties
+ */
+ @ApiModelProperty(
+ value = "Whether the controller service supports sensitive dynamic properties."
+ )
+ public Boolean getSupportsSensitiveDynamicProperties() {
+ return supportsSensitiveDynamicProperties;
+ }
+
+ public void setSupportsSensitiveDynamicProperties(final Boolean supportsSensitiveDynamicProperties) {
+ this.supportsSensitiveDynamicProperties = supportsSensitiveDynamicProperties;
+ }
+
/**
* @return The state of this controller service. Possible values are ENABLED, ENABLING, DISABLED, DISABLING
*/
@@ -244,6 +260,20 @@ public class ControllerServiceDTO extends ComponentDTO {
this.descriptors = descriptors;
}
+ /**
+ * @return Set of sensitive dynamic property names
+ */
+ @ApiModelProperty(
+ value = "Set of sensitive dynamic property names"
+ )
+ public Set<String> getSensitiveDynamicPropertyNames() {
+ return sensitiveDynamicPropertyNames;
+ }
+
+ public void setSensitiveDynamicPropertyNames(final Set<String> sensitiveDynamicPropertyNames) {
+ this.sensitiveDynamicPropertyNames = sensitiveDynamicPropertyNames;
+ }
+
/**
* @return the URL for this controller services custom configuration UI if applicable. Null otherwise
*/
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
index d6b8280f04..7fe1e689a7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java
@@ -30,6 +30,7 @@ public class ProcessorConfigDTO {
private Map<String, String> properties;
private Map<String, PropertyDescriptorDTO> descriptors;
+ private Set<String> sensitiveDynamicPropertyNames;
// settings
private String schedulingPeriod;
@@ -226,6 +227,20 @@ public class ProcessorConfigDTO {
this.descriptors = descriptors;
}
+ /**
+ * @return Set of sensitive dynamic property names
+ */
+ @ApiModelProperty(
+ value = "Set of sensitive dynamic property names"
+ )
+ public Set<String> getSensitiveDynamicPropertyNames() {
+ return sensitiveDynamicPropertyNames;
+ }
+
+ public void setSensitiveDynamicPropertyNames(final Set<String> sensitiveDynamicPropertyNames) {
+ this.sensitiveDynamicPropertyNames = sensitiveDynamicPropertyNames;
+ }
+
/**
* Annotation data for this processor.
*
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
index 04bae8c64a..78af13eb71 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java
@@ -42,6 +42,7 @@ public class ProcessorDTO extends ComponentDTO {
private Boolean supportsParallelProcessing;
private Boolean supportsEventDriven;
private Boolean supportsBatching;
+ private Boolean supportsSensitiveDynamicProperties;
private Boolean persistsState;
private Boolean restricted;
private Boolean deprecated;
@@ -150,6 +151,20 @@ public class ProcessorDTO extends ComponentDTO {
this.supportsParallelProcessing = supportsParallelProcessing;
}
+ /**
+ * @return whether this processor supports sensitive dynamic properties
+ */
+ @ApiModelProperty(
+ value = "Whether the processor supports sensitive dynamic properties."
+ )
+ public Boolean getSupportsSensitiveDynamicProperties() {
+ return supportsSensitiveDynamicProperties;
+ }
+
+ public void setSupportsSensitiveDynamicProperties(final Boolean supportsSensitiveDynamicProperties) {
+ this.supportsSensitiveDynamicProperties = supportsSensitiveDynamicProperties;
+ }
+
/**
* @return whether this processor persists state
*/
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java
index cccb50dce5..052885325c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java
@@ -21,6 +21,7 @@ import io.swagger.annotations.ApiModelProperty;
import javax.xml.bind.annotation.XmlType;
import java.util.Collection;
import java.util.Map;
+import java.util.Set;
/**
* Component that is capable of reporting internal NiFi state to an external service
@@ -41,6 +42,7 @@ public class ReportingTaskDTO extends ComponentDTO {
private Boolean deprecated;
private Boolean isExtensionMissing;
private Boolean multipleVersionsAvailable;
+ private Boolean supportsSensitiveDynamicProperties;
private String schedulingPeriod;
private String schedulingStrategy;
@@ -48,6 +50,7 @@ public class ReportingTaskDTO extends ComponentDTO {
private Map<String, String> properties;
private Map<String, PropertyDescriptorDTO> descriptors;
+ private Set<String> sensitiveDynamicPropertyNames;
private String customUiUrl;
private String annotationData;
@@ -200,6 +203,20 @@ public class ReportingTaskDTO extends ComponentDTO {
this.multipleVersionsAvailable = multipleVersionsAvailable;
}
+ /**
+ * @return whether this reporting task supports sensitive dynamic properties
+ */
+ @ApiModelProperty(
+ value = "Whether the reporting task supports sensitive dynamic properties."
+ )
+ public Boolean getSupportsSensitiveDynamicProperties() {
+ return supportsSensitiveDynamicProperties;
+ }
+
+ public void setSupportsSensitiveDynamicProperties(final Boolean supportsSensitiveDynamicProperties) {
+ this.supportsSensitiveDynamicProperties = supportsSensitiveDynamicProperties;
+ }
+
/**
* @return current scheduling state of the reporting task
*/
@@ -257,6 +274,20 @@ public class ReportingTaskDTO extends ComponentDTO {
this.descriptors = descriptors;
}
+ /**
+ * @return Set of sensitive dynamic property names
+ */
+ @ApiModelProperty(
+ value = "Set of sensitive dynamic property names"
+ )
+ public Set<String> getSensitiveDynamicPropertyNames() {
+ return sensitiveDynamicPropertyNames;
+ }
+
+ public void setSensitiveDynamicPropertyNames(final Set<String> sensitiveDynamicPropertyNames) {
+ this.sensitiveDynamicPropertyNames = sensitiveDynamicPropertyNames;
+ }
+
/**
* @return the URL for this reporting task custom configuration UI if applicable. Null otherwise
*/
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingContext.java
index 68e4004468..4acc0925c9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingContext.java
@@ -27,6 +27,7 @@ import org.apache.nifi.components.resource.StandardResourceContext;
import org.apache.nifi.components.resource.StandardResourceReferenceFactory;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.events.BulletinFactory;
@@ -44,7 +45,7 @@ import java.util.LinkedHashMap;
import java.util.Map;
public abstract class AbstractReportingContext implements ReportingContext {
- private final ReportingTask reportingTask;
+ private final ReportingTaskNode reportingTaskNode;
private final BulletinRepository bulletinRepository;
private final ControllerServiceProvider serviceProvider;
private final Map<PropertyDescriptor, String> properties;
@@ -52,14 +53,14 @@ public abstract class AbstractReportingContext implements ReportingContext {
private final ParameterLookup parameterLookup;
private final VariableRegistry variableRegistry;
- public AbstractReportingContext(final ReportingTask reportingTask, final BulletinRepository bulletinRepository,
+ public AbstractReportingContext(final ReportingTaskNode reportingTaskNode, final BulletinRepository bulletinRepository,
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider controllerServiceProvider,
final ParameterLookup parameterLookup, final VariableRegistry variableRegistry) {
this.bulletinRepository = bulletinRepository;
this.properties = Collections.unmodifiableMap(properties);
this.serviceProvider = controllerServiceProvider;
- this.reportingTask = reportingTask;
+ this.reportingTaskNode = reportingTaskNode;
this.parameterLookup = parameterLookup;
this.variableRegistry = variableRegistry;
this.preparedQueries = new HashMap<>();
@@ -77,7 +78,7 @@ public abstract class AbstractReportingContext implements ReportingContext {
}
protected ReportingTask getReportingTask() {
- return reportingTask;
+ return reportingTaskNode.getReportingTask();
}
@Override
@@ -101,7 +102,7 @@ public abstract class AbstractReportingContext implements ReportingContext {
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
- final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(property.getName());
+ final PropertyDescriptor descriptor = reportingTaskNode.getPropertyDescriptor(property.getName());
if (descriptor == null) {
return null;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index f767bfc32d..c64ed47ed8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -217,8 +217,8 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
}
@Override
- public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties) {
- super.setProperties(properties, allowRemovalOfRequiredProperties);
+ public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties, final Set<String> sensitiveDynamicPropertyNames) {
+ super.setProperties(properties, allowRemovalOfRequiredProperties, sensitiveDynamicPropertyNames);
// It's possible that changing the properties of this Controller Service could alter the Classloader Isolation Key of a referencing
// component so reload any referencing component as necessary.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
index 547265898c..04052be0a3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
@@ -59,6 +59,7 @@ import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.flow.VersionedPort;
import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedPropertyDescriptor;
import org.apache.nifi.flow.VersionedRemoteGroupPort;
import org.apache.nifi.flow.VersionedRemoteProcessGroup;
import org.apache.nifi.flow.VersionedReportingTask;
@@ -118,6 +119,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@@ -1195,8 +1197,9 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
service.setComments(proposed.getComments());
service.setName(proposed.getName());
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(service, proposed.getProperties(), proposed.getPropertyDescriptors().values());
final Map<String, String> properties = populatePropertiesMap(service, proposed.getProperties(), service.getProcessGroup());
- service.setProperties(properties, true);
+ service.setProperties(properties, true, sensitiveDynamicPropertyNames);
if (!isEqual(service.getBundleCoordinate(), proposed.getBundle())) {
final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
@@ -1209,6 +1212,35 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
}
}
+ private Set<String> getSensitiveDynamicPropertyNames(
+ final ComponentNode componentNode,
+ final Map<String, String> proposedProperties,
+ final Collection<VersionedPropertyDescriptor> proposedDescriptors
+ ) {
+ final Set<String> sensitiveDynamicPropertyNames = new LinkedHashSet<>();
+
+ // Find sensitive dynamic property names using proposed Versioned Property Descriptors
+ proposedDescriptors.stream()
+ .filter(VersionedPropertyDescriptor::isSensitive)
+ .map(VersionedPropertyDescriptor::getName)
+ .map(componentNode::getPropertyDescriptor)
+ .filter(PropertyDescriptor::isDynamic)
+ .map(PropertyDescriptor::getName)
+ .forEach(sensitiveDynamicPropertyNames::add);
+
+ // Find Encrypted Property values and find associated dynamic Property Descriptor names
+ proposedProperties.entrySet()
+ .stream()
+ .filter(entry -> isValueEncrypted(entry.getValue()))
+ .map(Map.Entry::getKey)
+ .map(componentNode::getPropertyDescriptor)
+ .filter(PropertyDescriptor::isDynamic)
+ .map(PropertyDescriptor::getName)
+ .forEach(sensitiveDynamicPropertyNames::add);
+
+ return sensitiveDynamicPropertyNames;
+ }
+
private Map<String, String> populatePropertiesMap(final ComponentNode componentNode, final Map<String, String> proposedProperties, final ProcessGroup group) {
// Explicitly set all existing properties to null, except for sensitive properties, so that if there isn't an entry in the proposedProperties
@@ -1237,16 +1269,13 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
// of an existing service that is outside the current processor group, and if it is we want to leave
// the property set to that value
String existingExternalServiceId = null;
- final PropertyDescriptor componentDescriptor = componentNode.getPropertyDescriptor(propertyName);
- if (componentDescriptor != null) {
- final String componentDescriptorValue = componentNode.getEffectivePropertyValue(componentDescriptor);
- if (componentDescriptorValue != null) {
- final ProcessGroup parentGroup = group.getParent();
- if (parentGroup != null) {
- final ControllerServiceNode serviceNode = parentGroup.findControllerService(componentDescriptorValue, false, true);
- if (serviceNode != null) {
- existingExternalServiceId = componentDescriptorValue;
- }
+ final String componentDescriptorValue = componentNode.getEffectivePropertyValue(descriptor);
+ if (componentDescriptorValue != null) {
+ final ProcessGroup parentGroup = group.getParent();
+ if (parentGroup != null) {
+ final ControllerServiceNode serviceNode = parentGroup.findControllerService(componentDescriptorValue, false, true);
+ if (serviceNode != null) {
+ existingExternalServiceId = componentDescriptorValue;
}
}
}
@@ -1270,9 +1299,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
// populated value. The exception to this rule is if the currently configured value is a Parameter Reference and the Versioned Flow is empty. In this case, it implies
// that the Versioned Flow has changed from a Parameter Reference to an explicit value. In this case, we do in fact want to change the value of the Sensitive Property from
// the current parameter reference to an unset value.
- final boolean sensitive = componentNode.getPropertyDescriptor(propertyName).isSensitive();
- if (sensitive && value == null) {
- final PropertyConfiguration propertyConfiguration = componentNode.getProperty(componentNode.getPropertyDescriptor(propertyName));
+ if (descriptor.isSensitive() && value == null) {
+ final PropertyConfiguration propertyConfiguration = componentNode.getProperty(descriptor);
if (propertyConfiguration == null) {
continue;
}
@@ -1295,23 +1323,21 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
}
private static String decrypt(final String value, final PropertyDecryptor decryptor) {
- if (value == null) {
- return null;
- }
- if (!value.startsWith(ENC_PREFIX)) {
- return value;
- }
- if (!value.endsWith(ENC_SUFFIX)) {
+ if (isValueEncrypted(value)) {
+ try {
+ return decryptor.decrypt(value.substring(ENC_PREFIX.length(), value.length() - ENC_SUFFIX.length()));
+ } catch (EncryptionException e) {
+ final String moreDescriptiveMessage = "There was a problem decrypting a sensitive flow configuration value. " +
+ "Check that the nifi.sensitive.props.key value in nifi.properties matches the value used to encrypt the flow.xml.gz file";
+ throw new EncryptionException(moreDescriptiveMessage, e);
+ }
+ } else {
return value;
}
+ }
- try {
- return decryptor.decrypt(value.substring(ENC_PREFIX.length(), value.length() - ENC_SUFFIX.length()));
- } catch (EncryptionException e) {
- final String moreDescriptiveMessage = "There was a problem decrypting a sensitive flow configuration value. " +
- "Check that the nifi.sensitive.props.key value in nifi.properties matches the value used to encrypt the flow.xml.gz file";
- throw new EncryptionException(moreDescriptiveMessage, e);
- }
+ private static boolean isValueEncrypted(final String value) {
+ return value != null && value.startsWith(ENC_PREFIX) && value.endsWith(ENC_SUFFIX);
}
private void verifyCanSynchronize(final ParameterContext parameterContext, final VersionedParameterContext proposed) throws FlowSynchronizationException {
@@ -2490,8 +2516,9 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
processor.setName(proposed.getName());
processor.setPenalizationPeriod(proposed.getPenaltyDuration());
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(processor, proposed.getProperties(), proposed.getPropertyDescriptors().values());
final Map<String, String> properties = populatePropertiesMap(processor, proposed.getProperties(), processor.getProcessGroup());
- processor.setProperties(properties, true);
+ processor.setProperties(properties, true, sensitiveDynamicPropertyNames);
processor.setRunDuration(proposed.getRunDurationMillis(), TimeUnit.MILLISECONDS);
processor.setSchedulingStrategy(SchedulingStrategy.valueOf(proposed.getSchedulingStrategy()));
processor.setScheduldingPeriod(proposed.getSchedulingPeriod());
@@ -3178,7 +3205,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(proposed.getSchedulingStrategy()));
reportingTask.setAnnotationData(proposed.getAnnotationData());
- reportingTask.setProperties(proposed.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(reportingTask, proposed.getProperties(), proposed.getPropertyDescriptors().values());
+ reportingTask.setProperties(proposed.getProperties(), false, sensitiveDynamicPropertyNames);
// enable/disable/start according to the ScheduledState
switch (proposed.getScheduledState()) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 1dcd79bac1..795db39b1c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -146,7 +146,7 @@ public class StandardProcessContext implements ProcessContext, ControllerService
}
// Get the "canonical" Property Descriptor from the Processor
- final PropertyDescriptor canonicalDescriptor = procNode.getProcessor().getPropertyDescriptor(descriptor.getName());
+ final PropertyDescriptor canonicalDescriptor = procNode.getPropertyDescriptor(descriptor.getName());
final String defaultValue = canonicalDescriptor.getDefaultValue();
return new StandardPropertyValue(resourceContext, defaultValue, this, procNode.getParameterLookup(), preparedQueries.get(descriptor), procNode.getVariableRegistry());
@@ -160,8 +160,7 @@ public class StandardProcessContext implements ProcessContext, ControllerService
@Override
public PropertyValue getProperty(final String propertyName) {
verifyTaskActive();
- final Processor processor = procNode.getProcessor();
- final PropertyDescriptor descriptor = processor.getPropertyDescriptor(propertyName);
+ final PropertyDescriptor descriptor = procNode.getPropertyDescriptor(propertyName);
if (descriptor == null) {
return null;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizerTest.java
index efd87f3eac..c768c4bf2e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizerTest.java
@@ -302,7 +302,7 @@ public class StandardVersionedComponentSynchronizerTest {
synchronizer.synchronize(processorA, versionedProcessor, group, synchronizationOptions);
// Ensure that the processor was updated as expected.
- verify(processorA).setProperties(versionedProcessor.getProperties(), true);
+ verify(processorA).setProperties(versionedProcessor.getProperties(), true, Collections.emptySet());
verify(processorA).setName(versionedProcessor.getName());
verify(componentScheduler, times(0)).startComponent(any(Connectable.class));
}
@@ -327,7 +327,7 @@ public class StandardVersionedComponentSynchronizerTest {
synchronizer.synchronize(processorA, versionedProcessor, group, synchronizationOptions);
verify(group, times(1)).stopProcessor(processorA);
- verify(processorA).setProperties(versionedProcessor.getProperties(), true);
+ verify(processorA).setProperties(versionedProcessor.getProperties(), true, Collections.emptySet());
verify(componentScheduler, atLeast(1)).startComponent(any(Connectable.class));
}
@@ -346,7 +346,7 @@ public class StandardVersionedComponentSynchronizerTest {
verifyStopped(processorA);
verifyNotRestarted(processorA);
verify(processorA, times(0)).terminate();
- verify(processorA, times(0)).setProperties(versionedProcessor.getProperties());
+ verify(processorA, times(0)).setProperties(eq(versionedProcessor.getProperties()), anyBoolean(), anySet());
verify(processorA, times(0)).setName(versionedProcessor.getName());
}
@@ -362,7 +362,7 @@ public class StandardVersionedComponentSynchronizerTest {
verifyStopped(processorA);
verifyRestarted(processorA);
verify(processorA, times(1)).terminate();
- verify(processorA, times(1)).setProperties(versionedProcessor.getProperties(), true);
+ verify(processorA, times(1)).setProperties(versionedProcessor.getProperties(), true, Collections.emptySet());
verify(processorA, times(1)).setName(versionedProcessor.getName());
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index 61d17eb4b8..c2f7636a7a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -105,6 +105,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
private final Lock lock = new ReentrantLock();
private final ConcurrentMap<PropertyDescriptor, PropertyConfiguration> properties = new ConcurrentHashMap<>();
+ private final AtomicReference<Set<String>> sensitiveDynamicPropertyNames = new AtomicReference<>(new HashSet<>());
private volatile String additionalResourcesFingerprint;
private final AtomicReference<ValidationState> validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList()));
private final ValidationTrigger validationTrigger;
@@ -228,14 +229,24 @@ public abstract class AbstractComponentNode implements ComponentNode {
return false;
}
+ /**
+ * Set Properties updates internal Map of Property Descriptors and values along with current definition of Sensitive Dynamic Property Names
+ *
+ * @param properties Property Names and Values to be updated
+ * @param allowRemovalOfRequiredProperties Allow Removal of Required Properties
+ * @param updatedSensitiveDynamicPropertyNames Requested Sensitive Dynamic Property Names replaces current configuration
+ */
@Override
- public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties) {
+ public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties, final Set<String> updatedSensitiveDynamicPropertyNames) {
if (properties == null) {
return;
}
lock.lock();
try {
+ Objects.requireNonNull(updatedSensitiveDynamicPropertyNames, "Sensitive Dynamic Property Names required");
+ sensitiveDynamicPropertyNames.getAndSet(updatedSensitiveDynamicPropertyNames);
+
verifyCanUpdateProperties(properties);
// Determine the Classloader Isolation Key, if applicable, so we can determine whether or not the key changes by setting properties.
@@ -247,8 +258,15 @@ public abstract class AbstractComponentNode implements ComponentNode {
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), id)) {
boolean classpathChanged = false;
for (final Map.Entry<String, String> entry : properties.entrySet()) {
+ final String propertyName = entry.getKey();
+
+ // Set sensitive status on dynamic properties after getting canonical representation of Property Descriptor
+ final PropertyDescriptor componentDescriptor = getComponent().getPropertyDescriptor(propertyName);
+ final PropertyDescriptor descriptor = componentDescriptor.isDynamic() && updatedSensitiveDynamicPropertyNames.contains(propertyName)
+ ? new PropertyDescriptor.Builder().fromPropertyDescriptor(componentDescriptor).sensitive(true).build()
+ : componentDescriptor;
+
// determine if any of the property changes require resetting the InstanceClassLoader
- final PropertyDescriptor descriptor = getComponent().getPropertyDescriptor(entry.getKey());
if (descriptor.isDynamicClasspathModifier()) {
classpathChanged = true;
}
@@ -260,21 +278,21 @@ public abstract class AbstractComponentNode implements ComponentNode {
}
}
- if (entry.getKey() != null && entry.getValue() == null) {
- removeProperty(entry.getKey(), allowRemovalOfRequiredProperties);
- } else if (entry.getKey() != null) {
+ if (propertyName != null && entry.getValue() == null) {
+ removeProperty(propertyName, allowRemovalOfRequiredProperties);
+ } else if (propertyName != null) {
// Use the EL-Agnostic Parameter Parser to gather the list of referenced Parameters. We do this because we want to to keep track of which parameters
// are referenced, regardless of whether or not they are referenced from within an EL Expression. However, we also will need to derive a different ParameterTokenList
// that we can provide to the PropertyConfiguration, so that when compiling the Expression Language Expressions, we are able to keep the Parameter Reference within
// the Expression's text.
- final PropertyConfiguration propertyConfiguration = configurationMap.get(entry.getKey());
+ final PropertyConfiguration propertyConfiguration = configurationMap.get(propertyName);
final List<ParameterReference> parameterReferences = propertyConfiguration.getParameterReferences();
for (final ParameterReference reference : parameterReferences) {
// increment count in map for this parameter
parameterReferenceCounts.merge(reference.getParameterName(), 1, (a, b) -> a == -1 ? null : a + b);
}
- setProperty(entry.getKey(), propertyConfiguration, this.properties::get);
+ setProperty(descriptor, propertyConfiguration, this.properties::get);
}
}
@@ -383,7 +401,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(PERFORM_VALIDATION_STEP_NAME)
.outcome(Outcome.FAILED)
- .explanation("Component is invalid: " + result.toString())
+ .explanation("Component is invalid: " + result)
.build());
}
@@ -442,12 +460,10 @@ public abstract class AbstractComponentNode implements ComponentNode {
}
// Keep setProperty/removeProperty private so that all calls go through setProperties
- private void setProperty(final String name, final PropertyConfiguration propertyConfiguration, final Function<PropertyDescriptor, PropertyConfiguration> valueToCompareFunction) {
- if (name == null || propertyConfiguration == null || propertyConfiguration.getRawValue() == null) {
- throw new IllegalArgumentException("Name or Value can not be null");
- }
+ private void setProperty(final PropertyDescriptor descriptor, final PropertyConfiguration propertyConfiguration, final Function<PropertyDescriptor, PropertyConfiguration> valueToCompareFunction) {
+ // Remove current PropertyDescriptor to force updated instance references
+ properties.remove(descriptor);
- final PropertyDescriptor descriptor = getComponent().getPropertyDescriptor(name);
final PropertyConfiguration propertyModComparisonValue = valueToCompareFunction.apply(descriptor);
final PropertyConfiguration oldConfiguration = properties.put(descriptor, propertyConfiguration);
final String effectiveValue = propertyConfiguration.getEffectiveValue(getParameterContext());
@@ -536,10 +552,12 @@ public abstract class AbstractComponentNode implements ComponentNode {
final Map<PropertyDescriptor, PropertyConfiguration> props = new LinkedHashMap<>();
for (final PropertyDescriptor descriptor : supported) {
- props.put(descriptor, null);
+ // Get Canonical Property Descriptor
+ props.put(getPropertyDescriptor(descriptor.getName()), null);
}
- props.putAll(properties);
+ // Get Canonical Property Descriptor for returned Map of properties
+ properties.forEach((descriptor, config) -> props.put(getPropertyDescriptor(descriptor.getName()), config));
return props;
}
}
@@ -603,11 +621,11 @@ public abstract class AbstractComponentNode implements ComponentNode {
continue;
}
- setProperty(propertyDescriptor.getName(), configuration, descriptor -> createPropertyConfiguration(descriptor.getDefaultValue(), descriptor.isExpressionLanguageSupported()));
+ setProperty(propertyDescriptor, configuration, descriptor -> createPropertyConfiguration(descriptor.getDefaultValue()));
}
}
- private PropertyConfiguration createPropertyConfiguration(final String value, final boolean supportsEL) {
+ private PropertyConfiguration createPropertyConfiguration(final String value) {
final ParameterParser parser = new ExpressionLanguageAwareParameterParser();
final ParameterTokenList references = parser.parseTokens(value);
final VariableImpact variableImpact = Query.prepare(value).getVariableImpact();
@@ -725,6 +743,16 @@ public abstract class AbstractComponentNode implements ComponentNode {
protected Collection<ValidationResult> computeValidationErrors(final ValidationContext validationContext) {
Throwable failureCause = null;
try {
+ if (!sensitiveDynamicPropertyNames.get().isEmpty() && !isSupportsSensitiveDynamicProperties()) {
+ return Collections.singletonList(
+ new ValidationResult.Builder()
+ .subject("Component")
+ .valid(false)
+ .explanation(String.format("Sensitive Dynamic Properties %s configured but not supported", sensitiveDynamicPropertyNames))
+ .build()
+ );
+ }
+
final List<ValidationResult> invalidParameterResults = validateParameterReferences(validationContext);
if (!invalidParameterResults.isEmpty()) {
// At this point, we are not able to properly resolve all property values, so we will not attempt to perform
@@ -971,7 +999,12 @@ public abstract class AbstractComponentNode implements ComponentNode {
@Override
public PropertyDescriptor getPropertyDescriptor(final String name) {
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
- return getComponent().getPropertyDescriptor(name);
+ final PropertyDescriptor propertyDescriptor = getComponent().getPropertyDescriptor(name);
+ if (propertyDescriptor.isDynamic() && sensitiveDynamicPropertyNames.get().contains(name)) {
+ return new PropertyDescriptor.Builder().fromPropertyDescriptor(propertyDescriptor).sensitive(true).build();
+ } else {
+ return propertyDescriptor;
+ }
}
}
@@ -1311,5 +1344,4 @@ public abstract class AbstractComponentNode implements ComponentNode {
}
protected abstract ParameterContext getParameterContext();
-
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
index 5316c1c621..136dac5fef 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result;
@@ -41,6 +42,7 @@ import org.apache.nifi.registry.ComponentVariableRegistry;
import java.net.URL;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -48,21 +50,21 @@ import java.util.concurrent.TimeUnit;
public interface ComponentNode extends ComponentAuthorizable {
@Override
- public String getIdentifier();
+ String getIdentifier();
- public String getName();
+ String getName();
- public void setName(String name);
+ void setName(String name);
- public String getAnnotationData();
+ String getAnnotationData();
- public void setAnnotationData(String data);
+ void setAnnotationData(String data);
- public default void setProperties(Map<String, String> properties) {
- setProperties(properties, false);
+ default void setProperties(Map<String, String> properties) {
+ setProperties(properties, false, Collections.emptySet());
}
- public void setProperties(Map<String, String> properties, boolean allowRemovalOfRequiredProperties);
+ void setProperties(Map<String, String> properties, boolean allowRemovalOfRequiresProperties, Set<String> sensitiveDynamicPropertyNames);
void verifyCanUpdateProperties(final Map<String, String> properties);
@@ -190,6 +192,15 @@ public interface ComponentNode extends ComponentAuthorizable {
*/
boolean isValidationNecessary();
+ /**
+ * Indicates whether the Component supports sensitive dynamic properties
+ *
+ * @return Support status for Sensitive Dynamic Properties
+ */
+ default boolean isSupportsSensitiveDynamicProperties() {
+ return getComponent().getClass().isAnnotationPresent(SupportsSensitiveDynamicProperties.class);
+ }
+
/**
* @return the variable registry for this component
*/
@@ -200,7 +211,7 @@ public interface ComponentNode extends ComponentAuthorizable {
*
* @return the processor's current Validation Status
*/
- public abstract ValidationStatus getValidationStatus();
+ ValidationStatus getValidationStatus();
/**
* Returns the processor's Validation Status, waiting up to the given amount of time for the Validation to complete
@@ -212,7 +223,7 @@ public interface ComponentNode extends ComponentAuthorizable {
* @param unit the time unit
* @return the ValidationStatus
*/
- public abstract ValidationStatus getValidationStatus(long timeout, TimeUnit unit);
+ ValidationStatus getValidationStatus(long timeout, TimeUnit unit);
/**
* Validates the component against the current configuration
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
index d43a450530..c5ccc80860 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
@@ -64,6 +64,10 @@ import static org.junit.Assert.assertTrue;
public class TestAbstractComponentNode {
+ private static final String PROPERTY_NAME = "abstract-property-name";
+
+ private static final String PROPERTY_VALUE = "abstract-property-value";
+
@Test(timeout = 5000)
public void testGetValidationStatusWithTimeout() {
final ValidationControlledAbstractComponentNode node = new ValidationControlledAbstractComponentNode(5000, Mockito.mock(ValidationTrigger.class));
@@ -107,7 +111,7 @@ public class TestAbstractComponentNode {
final Map<String, String> properties = new HashMap<>();
properties.put("abc", "#{abc}");
- node.setProperties(properties);
+ node.setProperties(properties, false, Collections.emptySet());
assertEquals(1, propertyModifications.size());
PropertyModification mod = propertyModifications.get(0);
@@ -150,7 +154,7 @@ public class TestAbstractComponentNode {
final Map<String, String> properties = new HashMap<>();
properties.put("abc", propertyValue);
node.verifyCanUpdateProperties(properties);
- node.setProperties(properties);
+ node.setProperties(properties, false, Collections.emptySet());
final ValidationContext validationContext = Mockito.mock(ValidationContext.class);
Mockito.when(validationContext.getProperties()).thenReturn(Collections.singletonMap(propertyDescriptor, propertyValue));
@@ -187,7 +191,7 @@ public class TestAbstractComponentNode {
node.pauseValidationTrigger();
for (int i = 0; i < 1000; i++) {
- node.setProperties(Collections.emptyMap());
+ node.setProperties(Collections.emptyMap(), false, Collections.emptySet());
assertEquals(0, validationCount.get());
}
node.resumeValidationTrigger();
@@ -224,6 +228,75 @@ public class TestAbstractComponentNode {
assertTrue("Enabling Service Validation Result not found", validationResult instanceof EnablingServiceValidationResult);
}
+ @Test
+ public void testSetProperties() {
+ final AbstractComponentNode node = new LocalComponentNode();
+
+ final PropertyDescriptor originalPropertyDescriptor = node.getPropertyDescriptor(PROPERTY_NAME);
+ assertTrue(originalPropertyDescriptor.isDynamic());
+ assertFalse(originalPropertyDescriptor.isSensitive());
+
+ final Map<String, String> properties = Collections.singletonMap(PROPERTY_NAME, PROPERTY_VALUE);
+ node.setProperties(properties);
+
+ final PropertyDescriptor updatedPropertyDescriptor = node.getPropertyDescriptor(PROPERTY_NAME);
+ assertTrue(updatedPropertyDescriptor.isDynamic());
+ assertFalse(updatedPropertyDescriptor.isSensitive());
+ }
+
+ @Test
+ public void testSetPropertiesSensitiveDynamicPropertyNames() {
+ final AbstractComponentNode node = new LocalComponentNode();
+
+ final Map<String, String> properties = Collections.singletonMap(PROPERTY_NAME, PROPERTY_VALUE);
+ final Set<String> sensitiveDynamicPropertyNames = Collections.singleton(PROPERTY_NAME);
+ node.setProperties(properties, false, sensitiveDynamicPropertyNames);
+
+ final PropertyDescriptor updatedPropertyDescriptor = node.getPropertyDescriptor(PROPERTY_NAME);
+ assertTrue(updatedPropertyDescriptor.isDynamic());
+ assertTrue(updatedPropertyDescriptor.isSensitive());
+
+ final Map<PropertyDescriptor, PropertyConfiguration> configuredProperties = node.getProperties();
+ final PropertyDescriptor configuredPropertyDescriptor = configuredProperties.keySet()
+ .stream()
+ .filter(descriptor -> descriptor.getName().equals(PROPERTY_NAME))
+ .findFirst()
+ .orElseThrow(() -> new IllegalStateException("Property Name not found"));
+ assertTrue(configuredPropertyDescriptor.isDynamic());
+ assertTrue(configuredPropertyDescriptor.isSensitive());
+
+ final PropertyConfiguration propertyConfiguration = configuredProperties.get(configuredPropertyDescriptor);
+ assertEquals(PROPERTY_VALUE, propertyConfiguration.getRawValue());
+ }
+
+ @Test
+ public void testSetPropertiesSensitiveDynamicPropertyNamesAddedRemoved() {
+ final AbstractComponentNode node = new LocalComponentNode();
+
+ final Map<String, String> properties = Collections.singletonMap(PROPERTY_NAME, PROPERTY_VALUE);
+ final Set<String> sensitiveDynamicPropertyNames = Collections.singleton(PROPERTY_NAME);
+ node.setProperties(properties, false, sensitiveDynamicPropertyNames);
+
+ final PropertyDescriptor sensitivePropertyDescriptor = node.getPropertyDescriptor(PROPERTY_NAME);
+ assertTrue(sensitivePropertyDescriptor.isDynamic());
+ assertTrue(sensitivePropertyDescriptor.isSensitive());
+
+ node.setProperties(properties, false, Collections.emptySet());
+
+ final PropertyDescriptor updatedPropertyDescriptor = node.getPropertyDescriptor(PROPERTY_NAME);
+ assertTrue(updatedPropertyDescriptor.isDynamic());
+ assertFalse(updatedPropertyDescriptor.isSensitive());
+
+ final Map<PropertyDescriptor, PropertyConfiguration> configuredProperties = node.getProperties();
+ final PropertyDescriptor configuredPropertyDescriptor = configuredProperties.keySet()
+ .stream()
+ .filter(descriptor -> descriptor.getName().equals(PROPERTY_NAME))
+ .findFirst()
+ .orElseThrow(() -> new IllegalStateException("Property Name not found"));
+ assertTrue(configuredPropertyDescriptor.isDynamic());
+ assertFalse(configuredPropertyDescriptor.isSensitive());
+ }
+
private ValidationContext getServiceValidationContext(final ControllerServiceState serviceState, final ControllerServiceProvider serviceProvider) {
final ValidationContext context = Mockito.mock(ValidationContext.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
index c1fb648868..6f4227d637 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSnippet.java
@@ -251,7 +251,8 @@ public class StandardFlowSnippet implements FlowSnippet {
for (final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices()) {
final String serviceId = controllerServiceDTO.getId();
final ControllerServiceNode serviceNode = flowManager.getControllerServiceNode(serviceId);
- serviceNode.setProperties(controllerServiceDTO.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = controllerServiceDTO.getSensitiveDynamicPropertyNames();
+ serviceNode.setProperties(controllerServiceDTO.getProperties(), false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
}
} finally {
serviceNodes.forEach(ControllerServiceNode::resumeValidationTrigger);
@@ -420,7 +421,8 @@ public class StandardFlowSnippet implements FlowSnippet {
group.addProcessor(procNode);
if (config.getProperties() != null) {
- procNode.setProperties(config.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = config.getSensitiveDynamicPropertyNames();
+ procNode.setProperties(config.getProperties(), false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
}
// Notify the processor node that the configuration (properties, e.g.) has been restored
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
index 6297f390c5..68aad4cfd3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
@@ -257,6 +257,7 @@ public class TemplateUtils {
processorDTO.setSupportsBatching(null);
processorDTO.setSupportsEventDriven(null);
processorDTO.setSupportsParallelProcessing(null);
+ processorDTO.setSupportsSensitiveDynamicProperties(null);
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
index ac61da95df..63d2b63a7c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
@@ -726,7 +726,8 @@ public class XmlFlowSynchronizer implements FlowSynchronizer {
reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(dto.getSchedulingStrategy()));
reportingTask.setAnnotationData(dto.getAnnotationData());
- reportingTask.setProperties(dto.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = dto.getSensitiveDynamicPropertyNames();
+ reportingTask.setProperties(dto.getProperties(), false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
return reportingTask;
} else {
// otherwise return the existing reporting task node
@@ -1269,7 +1270,8 @@ public class XmlFlowSynchronizer implements FlowSynchronizer {
procNode.setAutoTerminatedRelationships(relationships);
}
- procNode.setProperties(config.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = config.getSensitiveDynamicPropertyNames();
+ procNode.setProperties(config.getProperties(), false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
final ScheduledState scheduledState = ScheduledState.valueOf(processorDTO.getState());
if (ScheduledState.RUNNING.equals(scheduledState)) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java
index ac1d561071..bfa7eb68e2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java
@@ -20,13 +20,13 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.ReportingTask;
import java.util.Map;
@@ -37,9 +37,9 @@ public class StandardReportingContext extends AbstractReportingContext implement
private final boolean analyticsEnabled;
public StandardReportingContext(final FlowController flowController, final BulletinRepository bulletinRepository,
- final Map<PropertyDescriptor, String> properties, final ReportingTask reportingTask,
+ final Map<PropertyDescriptor, String> properties, final ReportingTaskNode reportingTaskNode,
final VariableRegistry variableRegistry, final ParameterLookup parameterLookup) {
- super(reportingTask, bulletinRepository, properties, flowController.getControllerServiceProvider(), parameterLookup, variableRegistry);
+ super(reportingTaskNode, bulletinRepository, properties, flowController.getControllerServiceProvider(), parameterLookup, variableRegistry);
this.flowController = flowController;
this.eventAccess = flowController.getEventAccess();
this.analyticsEnabled = flowController.getStatusAnalyticsEngine() != null;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
index c9b4da9420..9ba5f713d8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
@@ -74,7 +74,7 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
@Override
public Class<?> getComponentClass() {
- return getReportingContext().getClass();
+ return getReportingTask().getClass();
}
@Override
@@ -84,7 +84,7 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
@Override
public ReportingContext getReportingContext() {
- return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getEffectivePropertyValues(), getReportingTask(), getVariableRegistry(), ParameterLookup.EMPTY);
+ return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getEffectivePropertyValues(), this, getVariableRegistry(), ParameterLookup.EMPTY);
}
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
index 01727c7705..838f765478 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
@@ -121,6 +121,7 @@ public class FlowFromDOMFactory {
final boolean enabled = getBoolean(element, "enabled");
dto.setState(enabled ? ControllerServiceState.ENABLED.name() : ControllerServiceState.DISABLED.name());
+ dto.setSensitiveDynamicPropertyNames(getSensitivePropertyNames(element));
dto.setProperties(getProperties(element, encryptor, flowEncodingVersion));
dto.setAnnotationData(getString(element, "annotationData"));
@@ -139,6 +140,7 @@ public class FlowFromDOMFactory {
dto.setState(getString(element, "scheduledState"));
dto.setSchedulingStrategy(getString(element, "schedulingStrategy"));
+ dto.setSensitiveDynamicPropertyNames(getSensitivePropertyNames(element));
dto.setProperties(getProperties(element, encryptor, flowEncodingVersion));
dto.setAnnotationData(getString(element, "annotationData"));
@@ -539,6 +541,7 @@ public class FlowFromDOMFactory {
configDto.setRunDurationMillis(TimeUnit.NANOSECONDS.toMillis(runDurationNanos));
}
+ configDto.setSensitiveDynamicPropertyNames(getSensitivePropertyNames(element));
configDto.setProperties(getProperties(element, encryptor, flowEncodingVersion));
configDto.setAnnotationData(getString(element, "annotationData"));
@@ -552,6 +555,21 @@ public class FlowFromDOMFactory {
return dto;
}
+ private static Set<String> getSensitivePropertyNames(final Element element) {
+ final Set<String> sensitivePropertyNames = new LinkedHashSet<>();
+
+ final List<Element> propertyElements = getChildrenByTagName(element, "property");
+ for (final Element propertyElement : propertyElements) {
+ final String rawPropertyValue = getString(propertyElement, "value");
+ if (isValueSensitive(rawPropertyValue)) {
+ final String name = getString(propertyElement, "name");
+ sensitivePropertyNames.add(name);
+ }
+ }
+
+ return sensitivePropertyNames;
+ }
+
private static LinkedHashMap<String, String> getProperties(final Element element, final PropertyEncryptor encryptor, final FlowEncodingVersion flowEncodingVersion) {
final LinkedHashMap<String, String> properties = new LinkedHashMap<>();
final List<Element> propertyNodeList = getChildrenByTagName(element, "property");
@@ -636,7 +654,7 @@ public class FlowFromDOMFactory {
}
private static String decrypt(final String value, final PropertyEncryptor encryptor) {
- if (value != null && value.startsWith(FlowSerializer.ENC_PREFIX) && value.endsWith(FlowSerializer.ENC_SUFFIX)) {
+ if (isValueSensitive(value)) {
try {
return encryptor.decrypt(value.substring(FlowSerializer.ENC_PREFIX.length(), value.length() - FlowSerializer.ENC_SUFFIX.length()));
} catch (EncryptionException e) {
@@ -649,4 +667,8 @@ public class FlowFromDOMFactory {
return value;
}
}
+
+ private static boolean isValueSensitive(final String value) {
+ return value != null && value.startsWith(FlowSerializer.ENC_PREFIX) && value.endsWith(FlowSerializer.ENC_SUFFIX);
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
index 10ac68f044..7f223b6449 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
@@ -24,6 +24,7 @@ import org.apache.nifi.authorization.ManagedAuthorizer;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
+import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.AbstractComponentNode;
@@ -52,12 +53,14 @@ import org.apache.nifi.encrypt.PropertyEncryptor;
import org.apache.nifi.flow.Bundle;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedConfigurableExtension;
import org.apache.nifi.flow.VersionedControllerService;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.flow.VersionedParameter;
import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedPropertyDescriptor;
import org.apache.nifi.flow.VersionedReportingTask;
import org.apache.nifi.groups.AbstractComponentScheduler;
import org.apache.nifi.groups.BundleUpdateStrategy;
@@ -105,6 +108,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -383,8 +387,7 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
final FlowComparator flowComparator = new StandardFlowComparator(localDataFlow, clusterDataFlow, Collections.emptySet(),
differenceDescriptor, encryptor::decrypt, VersionedComponent::getInstanceIdentifier);
- final FlowComparison flowComparison = flowComparator.compare();
- return flowComparison;
+ return flowComparator.compare();
}
private <T> Set<T> toSet(final List<T> values) {
@@ -499,8 +502,9 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
taskNode.setAnnotationData(reportingTask.getAnnotationData());
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(taskNode, reportingTask);
final Map<String, String> decryptedProperties = decryptProperties(reportingTask.getProperties());
- taskNode.setProperties(decryptedProperties);
+ taskNode.setProperties(decryptedProperties, false, sensitiveDynamicPropertyNames);
// enable/disable/start according to the ScheduledState
switch (reportingTask.getScheduledState()) {
@@ -721,13 +725,41 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
serviceNode.setAnnotationData(versionedControllerService.getAnnotationData());
serviceNode.setComments(versionedControllerService.getComments());
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(serviceNode, versionedControllerService);
final Map<String, String> decryptedProperties = decryptProperties(versionedControllerService.getProperties());
- serviceNode.setProperties(decryptedProperties);
+ serviceNode.setProperties(decryptedProperties, false, sensitiveDynamicPropertyNames);
} finally {
serviceNode.resumeValidationTrigger();
}
}
+ private Set<String> getSensitiveDynamicPropertyNames(final ComponentNode componentNode, final VersionedConfigurableExtension extension) {
+ final Set<String> versionedSensitivePropertyNames = new LinkedHashSet<>();
+
+ // Get Sensitive Property Names based on encrypted values including both supported and dynamic properties
+ extension.getProperties()
+ .entrySet()
+ .stream()
+ .filter(entry -> isValueSensitive(entry.getValue()))
+ .map(Map.Entry::getKey)
+ .forEach(versionedSensitivePropertyNames::add);
+
+ // Get Sensitive Property Names based on supported and dynamic property descriptors
+ extension.getPropertyDescriptors()
+ .values()
+ .stream()
+ .filter(VersionedPropertyDescriptor::isSensitive)
+ .map(VersionedPropertyDescriptor::getName)
+ .forEach(versionedSensitivePropertyNames::add);
+
+ // Filter combined Sensitive Property Names based on Component Property Descriptor status
+ return versionedSensitivePropertyNames.stream()
+ .map(componentNode::getPropertyDescriptor)
+ .filter(PropertyDescriptor::isDynamic)
+ .map(PropertyDescriptor::getName)
+ .collect(Collectors.toSet());
+ }
+
private Map<String, String> decryptProperties(final Map<String, String> encrypted) {
final Map<String, String> decrypted = new HashMap<>(encrypted.size());
encrypted.forEach((key, value) -> decrypted.put(key, decrypt(value)));
@@ -735,7 +767,7 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
}
private String decrypt(final String value) {
- if (value != null && value.startsWith(FlowSerializer.ENC_PREFIX) && value.endsWith(FlowSerializer.ENC_SUFFIX)) {
+ if (isValueSensitive(value)) {
try {
return encryptor.decrypt(value.substring(FlowSerializer.ENC_PREFIX.length(), value.length() - FlowSerializer.ENC_SUFFIX.length()));
} catch (EncryptionException e) {
@@ -749,6 +781,10 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
}
}
+ private boolean isValueSensitive(final String value) {
+ return value != null && value.startsWith(FlowSerializer.ENC_PREFIX) && value.endsWith(FlowSerializer.ENC_SUFFIX);
+ }
+
private BundleCoordinate createBundleCoordinate(final Bundle bundle, final String componentType) {
BundleCoordinate coordinate;
try {
@@ -957,8 +993,7 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
FileUtils.copy(gzipIn, baos);
- final byte[] contents = baos.toByteArray();
- return contents;
+ return baos.toByteArray();
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
index 30261cf13b..1907fe7a98 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
@@ -216,7 +216,8 @@ public class ControllerServiceLoader {
node.pauseValidationTrigger();
try {
node.setAnnotationData(dto.getAnnotationData());
- node.setProperties(dto.getProperties());
+ final Set<String> sensitiveDynamicPropertyNames = dto.getSensitiveDynamicPropertyNames();
+ node.setProperties(dto.getProperties(), false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
} finally {
node.resumeValidationTrigger();
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
index 60d19fdf4c..df764e5b5a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
@@ -44,6 +44,7 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
@@ -119,6 +120,9 @@ public class ControllerServiceAuditor extends NiFiAuditor {
// ensure the user was found
if (user != null) {
+ final Set<String> sensitiveDynamicPropertyNames = controllerServiceDTO.getSensitiveDynamicPropertyNames() == null
+ ? Collections.emptySet() : controllerServiceDTO.getSensitiveDynamicPropertyNames();
+
// determine the updated values
Map<String, String> updatedValues = extractConfiguredPropertyValues(controllerService, controllerServiceDTO);
@@ -144,13 +148,14 @@ public class ControllerServiceAuditor extends NiFiAuditor {
// create a configuration action accordingly
if (operation != null) {
// clear the value if this property is sensitive
- final PropertyDescriptor propertyDescriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property);
- if (propertyDescriptor != null && propertyDescriptor.isSensitive()) {
+ final PropertyDescriptor propertyDescriptor = controllerService.getPropertyDescriptor(property);
+ // Evaluate both Property Descriptor status and whether the client requested a new Sensitive Dynamic Property
+ if (propertyDescriptor != null && (propertyDescriptor.isSensitive() || sensitiveDynamicPropertyNames.contains(property))) {
if (newValue != null) {
- newValue = "********";
+ newValue = SENSITIVE_VALUE_PLACEHOLDER;
}
if (oldValue != null) {
- oldValue = "********";
+ oldValue = SENSITIVE_VALUE_PLACEHOLDER;
}
} else if (ANNOTATION_DATA.equals(property)) {
if (newValue != null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
index 8dd7c85739..7e3a3470d9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
@@ -35,6 +35,8 @@ import java.util.Collection;
*/
public abstract class NiFiAuditor {
+ protected static final String SENSITIVE_VALUE_PLACEHOLDER = "********";
+
private AuditService auditService;
private NiFiServiceFacade serviceFacade;
private ProcessGroupDAO processGroupDAO;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
index f9955eaa96..3522abad57 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
@@ -142,6 +142,8 @@ public class ProcessorAuditor extends NiFiAuditor {
// ensure the user was found
if (user != null) {
+ final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(processorDTO);
+
// determine the updated values
Map<String, String> updatedValues = extractConfiguredPropertyValues(processor, processorDTO);
@@ -167,13 +169,14 @@ public class ProcessorAuditor extends NiFiAuditor {
// create a configuration action accordingly
if (operation != null) {
// clear the value if this property is sensitive
- final PropertyDescriptor propertyDescriptor = processor.getProcessor().getPropertyDescriptor(property);
- if (propertyDescriptor != null && propertyDescriptor.isSensitive()) {
+ final PropertyDescriptor propertyDescriptor = processor.getPropertyDescriptor(property);
+ // Evaluate both Property Descriptor status and whether the client requested a new Sensitive Dynamic Property
+ if (propertyDescriptor != null && (propertyDescriptor.isSensitive() || sensitiveDynamicPropertyNames.contains(property))) {
if (newValue != null) {
- newValue = "********";
+ newValue = SENSITIVE_VALUE_PLACEHOLDER;
}
if (oldValue != null) {
- oldValue = "********";
+ oldValue = SENSITIVE_VALUE_PLACEHOLDER;
}
} else if (ANNOTATION_DATA.equals(property)) {
if (newValue != null && oldValue != null) {
@@ -354,6 +357,14 @@ public class ProcessorAuditor extends NiFiAuditor {
return action;
}
+ private Set<String> getSensitiveDynamicPropertyNames(final ProcessorDTO processorDTO) {
+ final ProcessorConfigDTO config = processorDTO.getConfig();
+ if (config == null) {
+ return Collections.emptySet();
+ }
+ return config.getSensitiveDynamicPropertyNames() == null ? Collections.emptySet() : config.getSensitiveDynamicPropertyNames();
+ }
+
/**
* Extracts the values for the configured properties from the specified Processor.
*/
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
index c83bc82f26..672b0313d1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
@@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
@@ -113,6 +114,9 @@ public class ReportingTaskAuditor extends NiFiAuditor {
// ensure the user was found
if (user != null) {
+ final Set<String> sensitiveDynamicPropertyNames = reportingTaskDTO.getSensitiveDynamicPropertyNames() == null
+ ? Collections.emptySet() : reportingTaskDTO.getSensitiveDynamicPropertyNames();
+
// determine the updated values
Map<String, String> updatedValues = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO);
@@ -138,13 +142,14 @@ public class ReportingTaskAuditor extends NiFiAuditor {
// create a configuration action accordingly
if (operation != null) {
// clear the value if this property is sensitive
- final PropertyDescriptor propertyDescriptor = reportingTask.getReportingTask().getPropertyDescriptor(property);
- if (propertyDescriptor != null && propertyDescriptor.isSensitive()) {
+ final PropertyDescriptor propertyDescriptor = reportingTask.getPropertyDescriptor(property);
+ // Evaluate both Property Descriptor status and whether the client requested a new Sensitive Dynamic Property
+ if (propertyDescriptor != null && (propertyDescriptor.isSensitive() || sensitiveDynamicPropertyNames.contains(property))) {
if (newValue != null) {
- newValue = "********";
+ newValue = SENSITIVE_VALUE_PLACEHOLDER;
}
if (oldValue != null) {
- oldValue = "********";
+ oldValue = SENSITIVE_VALUE_PLACEHOLDER;
}
} else if (ANNOTATION_DATA.equals(property)) {
if (newValue != null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 212ae50f5f..ff96408805 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -4513,7 +4513,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public PropertyDescriptorDTO getControllerServicePropertyDescriptor(final String id, final String property) {
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
- PropertyDescriptor descriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property);
+ PropertyDescriptor descriptor = controllerService.getPropertyDescriptor(property);
// return an invalid descriptor if the controller service doesn't support this property
if (descriptor == null) {
@@ -4557,7 +4557,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public PropertyDescriptorDTO getReportingTaskPropertyDescriptor(final String id, final String property) {
final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(id);
- PropertyDescriptor descriptor = reportingTask.getReportingTask().getPropertyDescriptor(property);
+ PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(property);
// return an invalid descriptor if the reporting task doesn't support this property
if (descriptor == null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
index a229f00072..e22a672b4f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
@@ -257,7 +257,13 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The property name to return the descriptor for.",
required = true
)
- @QueryParam("propertyName") final String propertyName) {
+ @QueryParam("propertyName") final String propertyName,
+ @ApiParam(
+ value = "Property Descriptor requested sensitive status",
+ defaultValue = "false"
+ )
+ @QueryParam("sensitive") final boolean sensitive
+ ) {
// ensure the property name is specified
if (propertyName == null) {
@@ -277,6 +283,11 @@ public class ControllerServiceResource extends ApplicationResource {
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getControllerServicePropertyDescriptor(id, propertyName);
+ // Adjust sensitive status for dynamic properties when sensitive status enabled
+ if (descriptor.isDynamic() && sensitive) {
+ descriptor.setSensitive(true);
+ }
+
// generate the response entity
final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
entity.setPropertyDescriptor(descriptor);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
index 49052b4051..f16c28ff31 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
@@ -403,7 +403,13 @@ public class ProcessorResource extends ApplicationResource {
value = "The property name.",
required = true
)
- @QueryParam("propertyName") final String propertyName) throws InterruptedException {
+ @QueryParam("propertyName") final String propertyName,
+ @ApiParam(
+ value = "Property Descriptor requested sensitive status",
+ defaultValue = "false"
+ )
+ @QueryParam("sensitive") final boolean sensitive
+ ) throws InterruptedException {
// ensure the property name is specified
if (propertyName == null) {
@@ -423,6 +429,11 @@ public class ProcessorResource extends ApplicationResource {
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getProcessorPropertyDescriptor(id, propertyName);
+ // Adjust sensitive status for dynamic properties when sensitive status enabled
+ if (descriptor.isDynamic() && sensitive) {
+ descriptor.setSensitive(true);
+ }
+
// generate the response entity
final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
entity.setPropertyDescriptor(descriptor);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
index 6f27ec5c32..3f32e6b1fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
@@ -243,7 +243,13 @@ public class ReportingTaskResource extends ApplicationResource {
value = "The property name.",
required = true
)
- @QueryParam("propertyName") final String propertyName) {
+ @QueryParam("propertyName") final String propertyName,
+ @ApiParam(
+ value = "Property Descriptor requested sensitive status",
+ defaultValue = "false"
+ )
+ @QueryParam("sensitive") final boolean sensitive
+ ) {
// ensure the property name is specified
if (propertyName == null) {
@@ -263,6 +269,11 @@ public class ReportingTaskResource extends ApplicationResource {
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getReportingTaskPropertyDescriptor(id, propertyName);
+ // Adjust sensitive status for dynamic properties when sensitive status enabled
+ if (descriptor.isDynamic() && sensitive) {
+ descriptor.setSensitive(true);
+ }
+
// generate the response entity
final PropertyDescriptorEntity entity = new PropertyDescriptorEntity();
entity.setPropertyDescriptor(descriptor);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 56b276e3e6..724fef5df7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1533,6 +1533,8 @@ public final class DtoFactory {
return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
}).collect(Collectors.toList());
+ final Class<? extends ReportingTask> reportingTaskClass = reportingTaskNode.getReportingTask().getClass();
+
final ReportingTaskDTO dto = new ReportingTaskDTO();
dto.setId(reportingTaskNode.getIdentifier());
dto.setName(reportingTaskNode.getName());
@@ -1544,7 +1546,8 @@ public final class DtoFactory {
dto.setActiveThreadCount(reportingTaskNode.getActiveThreadCount());
dto.setAnnotationData(reportingTaskNode.getAnnotationData());
dto.setComments(reportingTaskNode.getComments());
- dto.setPersistsState(reportingTaskNode.getReportingTask().getClass().isAnnotationPresent(Stateful.class));
+ dto.setPersistsState(reportingTaskClass.isAnnotationPresent(Stateful.class));
+ dto.setSupportsSensitiveDynamicProperties(reportingTaskNode.isSupportsSensitiveDynamicProperties());
dto.setRestricted(reportingTaskNode.isRestricted());
dto.setDeprecated(reportingTaskNode.isDeprecated());
dto.setExtensionMissing(reportingTaskNode.isExtensionMissing());
@@ -1620,17 +1623,20 @@ public final class DtoFactory {
return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
}).collect(Collectors.toList());
+ final Class<? extends ControllerService> controllerServiceClass = controllerServiceNode.getControllerServiceImplementation().getClass();
+
final ControllerServiceDTO dto = new ControllerServiceDTO();
dto.setId(controllerServiceNode.getIdentifier());
dto.setParentGroupId(controllerServiceNode.getProcessGroup() == null ? null : controllerServiceNode.getProcessGroup().getIdentifier());
dto.setName(controllerServiceNode.getName());
dto.setType(controllerServiceNode.getCanonicalClassName());
dto.setBundle(createBundleDto(bundleCoordinate));
- dto.setControllerServiceApis(createControllerServiceApiDto(controllerServiceNode.getControllerServiceImplementation().getClass()));
+ dto.setControllerServiceApis(createControllerServiceApiDto(controllerServiceClass));
dto.setState(controllerServiceNode.getState().name());
dto.setAnnotationData(controllerServiceNode.getAnnotationData());
dto.setComments(controllerServiceNode.getComments());
- dto.setPersistsState(controllerServiceNode.getControllerServiceImplementation().getClass().isAnnotationPresent(Stateful.class));
+ dto.setPersistsState(controllerServiceClass.isAnnotationPresent(Stateful.class));
+ dto.setSupportsSensitiveDynamicProperties(controllerServiceNode.isSupportsSensitiveDynamicProperties());
dto.setRestricted(controllerServiceNode.isRestricted());
dto.setDeprecated(controllerServiceNode.isDeprecated());
dto.setExtensionMissing(controllerServiceNode.isExtensionMissing());
@@ -3131,13 +3137,16 @@ public final class DtoFactory {
}
}
+ final Class<? extends Processor> processorClass = node.getProcessor().getClass();
+
final ProcessorDTO dto = new ProcessorDTO();
dto.setId(node.getIdentifier());
dto.setPosition(createPositionDto(node.getPosition()));
dto.setStyle(node.getStyle());
dto.setParentGroupId(node.getProcessGroup().getIdentifier());
dto.setInputRequirement(node.getInputRequirement().name());
- dto.setPersistsState(node.getProcessor().getClass().isAnnotationPresent(Stateful.class));
+ dto.setPersistsState(processorClass.isAnnotationPresent(Stateful.class));
+ dto.setSupportsSensitiveDynamicProperties(node.isSupportsSensitiveDynamicProperties());
dto.setRestricted(node.isRestricted());
dto.setDeprecated(node.isDeprecated());
dto.setExecutionNodeRestricted(node.isExecutionNodeRestricted());
@@ -4120,6 +4129,7 @@ public final class DtoFactory {
copy.setParentGroupId(original.getParentGroupId());
copy.setName(original.getName());
copy.setProperties(copy(original.getProperties()));
+ copy.setSensitiveDynamicPropertyNames(copy(original.getSensitiveDynamicPropertyNames()));
copy.setReferencingComponents(copy(original.getReferencingComponents()));
copy.setState(original.getState());
copy.setType(original.getType());
@@ -4203,6 +4213,7 @@ public final class DtoFactory {
copy.setSupportsParallelProcessing(original.getSupportsParallelProcessing());
copy.setSupportsEventDriven(original.getSupportsEventDriven());
copy.setSupportsBatching(original.getSupportsBatching());
+ copy.setSupportsSensitiveDynamicProperties(original.getSupportsSensitiveDynamicProperties());
copy.setPersistsState(original.getPersistsState());
copy.setExecutionNodeRestricted(original.isExecutionNodeRestricted());
copy.setExtensionMissing(original.getExtensionMissing());
@@ -4225,6 +4236,7 @@ public final class DtoFactory {
copy.setCustomUiUrl(original.getCustomUiUrl());
copy.setDescriptors(copy(original.getDescriptors()));
copy.setProperties(copy(original.getProperties()));
+ copy.setSensitiveDynamicPropertyNames(copy(original.getSensitiveDynamicPropertyNames()));
copy.setSchedulingPeriod(original.getSchedulingPeriod());
copy.setPenaltyDuration(original.getPenaltyDuration());
copy.setYieldDuration(original.getYieldDuration());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
index 3210347b22..5d37af3fcf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
@@ -369,7 +369,8 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
controllerService.setComments(comments);
}
if (isNotNull(properties)) {
- controllerService.setProperties(properties);
+ final Set<String> sensitiveDynamicPropertyNames = controllerServiceDTO.getSensitiveDynamicPropertyNames();
+ controllerService.setProperties(properties, false, sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
}
} finally {
controllerService.resumeValidationTrigger();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
index baf49f5ea1..fe8a02c2be 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
@@ -61,6 +61,7 @@ import org.slf4j.LoggerFactory;
import java.net.URL;
import java.text.ParseException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -198,7 +199,8 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
processor.setLossTolerant(config.isLossTolerant());
}
if (isNotNull(configProperties)) {
- processor.setProperties(configProperties);
+ final Set<String> sensitiveDynamicPropertyNames = config.getSensitiveDynamicPropertyNames();
+ processor.setProperties(configProperties, false,sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
}
if (isNotNull(retryCount)) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
index 74d5318753..cc89982c30 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
@@ -53,6 +53,7 @@ import org.quartz.CronExpression;
import java.net.URL;
import java.text.ParseException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -384,7 +385,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
reportingTask.setComments(comments);
}
if (isNotNull(properties)) {
- reportingTask.setProperties(properties);
+ final Set<String> sensitiveDynamicPropertyNames = reportingTaskDTO.getSensitiveDynamicPropertyNames();
+ reportingTask.setProperties(properties, false,sensitiveDynamicPropertyNames == null ? Collections.emptySet() : sensitiveDynamicPropertyNames);
}
} finally {
reportingTask.resumeValidationTrigger();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java
index 46b6b7e736..41c671737d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java
@@ -39,6 +39,7 @@ import org.apache.nifi.web.dao.SnippetDAO;
import org.apache.nifi.web.util.SnippetUtils;
import java.util.HashMap;
+import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@@ -293,12 +294,19 @@ public class StandardSnippetDAO implements SnippetDAO {
throw new IllegalArgumentException(String.format("Unable to create snippet because Processor '%s' could not be found", processorDTO.getId()));
}
+ // populate Sensitive Dynamic Property Names
+ final Set<String> sensitiveDynamicPropertyNames = new LinkedHashSet<>();
+ processorConfig.setSensitiveDynamicPropertyNames(sensitiveDynamicPropertyNames);
+
// look for sensitive properties get the actual value
for (Entry<PropertyDescriptor, String> entry : processorNode.getRawPropertyValues().entrySet()) {
final PropertyDescriptor descriptor = entry.getKey();
if (descriptor.isSensitive()) {
processorProperties.put(descriptor.getName(), entry.getValue());
+ if (descriptor.isDynamic()) {
+ sensitiveDynamicPropertyNames.add(descriptor.getName());
+ }
}
}
}
@@ -318,13 +326,25 @@ public class StandardSnippetDAO implements SnippetDAO {
throw new IllegalArgumentException(String.format("Unable to create snippet because Controller Service '%s' could not be found", serviceDTO.getId()));
}
- // look for sensitive properties get the actual value
- for (Entry<PropertyDescriptor, String> entry : serviceNode.getRawPropertyValues().entrySet()) {
- final PropertyDescriptor descriptor = entry.getKey();
+ final Set<String> sensitiveDynamicPropertyNames = new LinkedHashSet<>();
+ serviceDTO.setSensitiveDynamicPropertyNames(sensitiveDynamicPropertyNames);
+ putSensitiveProperties(serviceNode.getRawPropertyValues(), serviceProperties, sensitiveDynamicPropertyNames);
+ }
+ }
+ }
- if (descriptor.isSensitive()) {
- serviceProperties.put(descriptor.getName(), entry.getValue());
- }
+ private void putSensitiveProperties(
+ final Map<PropertyDescriptor, String> componentPropertyValues,
+ final Map<String, String> componentProperties,
+ final Set<String> sensitiveDynamicPropertyNames
+ ) {
+ for (Entry<PropertyDescriptor, String> entry : componentPropertyValues.entrySet()) {
+ final PropertyDescriptor descriptor = entry.getKey();
+
+ if (descriptor.isSensitive()) {
+ componentProperties.put(descriptor.getName(), entry.getValue());
+ if (descriptor.isDynamic()) {
+ sensitiveDynamicPropertyNames.add(descriptor.getName());
}
}
}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index 7bdca5f937..839d0f5188 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -80,6 +80,7 @@ import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -116,6 +117,7 @@ import org.apache.nifi.stream.io.StreamUtils;
import static org.apache.commons.lang3.StringUtils.trimToEmpty;
+@SupportsSensitiveDynamicProperties
@SupportsBatching
@Tags({"http", "https", "rest", "client"})
@InputRequirement(Requirement.INPUT_ALLOWED)
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/DifferenceType.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/DifferenceType.java
index fa89507507..077bec2127 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/DifferenceType.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/DifferenceType.java
@@ -128,6 +128,11 @@ public enum DifferenceType {
*/
PROPERTY_REMOVED("Property Removed"),
+ /**
+ * The sensitive status of the property changed
+ */
+ PROPERTY_SENSITIVITY_CHANGED("Property Sensitivity Changed"),
+
/**
* Property is unset or set to an explicit value in Flow A but set to (exactly) a parameter reference in Flow B. Note that if Flow A
* has a property set to "#{param1} abc" and it is changed to "#{param1} abc #{param2}" this would indicate a Difference Type of @{link #PROPERTY_CHANGED}, not
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
index 8c22b4d8b5..5fff2e84fc 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
@@ -291,9 +291,12 @@ public class StandardFlowComparator implements FlowComparator {
final String valueB = decrypt(rawValueB, descriptorsB.get(key));
final String valueA = decrypt(rawValueA, descriptorsA.get(key));
- VersionedPropertyDescriptor descriptor = descriptorsA.get(key);
+ final VersionedPropertyDescriptor descriptorA = descriptorsA.get(key);
+ final VersionedPropertyDescriptor descriptorB = descriptorsB.get(key);
+
+ VersionedPropertyDescriptor descriptor = descriptorA;
if (descriptor == null) {
- descriptor = descriptorsB.get(key);
+ descriptor = descriptorB;
}
final String displayName;
@@ -303,6 +306,12 @@ public class StandardFlowComparator implements FlowComparator {
displayName = descriptor.getDisplayName() == null ? descriptor.getName() : descriptor.getDisplayName();
}
+ if (descriptorA != null && descriptorB != null) {
+ if (descriptorA.isSensitive() != descriptorB.isSensitive()) {
+ differences.add(difference(DifferenceType.PROPERTY_SENSITIVITY_CHANGED, componentA, componentB, key, displayName, descriptorA.isSensitive(), descriptorB.isSensitive()));
+ }
+ }
+
if (valueA == null && valueB != null) {
if (isParameterReference(valueB)) {
differences.add(difference(DifferenceType.PROPERTY_PARAMETERIZED, componentA, componentB, key, displayName, null, null));
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
index 20bc8c607e..041542419f 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
@@ -53,6 +53,9 @@ public class StaticDifferenceDescriptor implements DifferenceDescriptor {
case PROPERTY_CHANGED:
description = String.format("Property '%s' for %s with ID %s is different", fieldName, componentA.getComponentType().getTypeName(), getId(componentA));
break;
+ case PROPERTY_SENSITIVITY_CHANGED:
+ description = String.format("Property '%s' for %s with ID %s has a different sensitive status", fieldName, componentA.getComponentType().getTypeName(), componentA.getIdentifier());
+ break;
case PROPERTY_PARAMETERIZED:
description = String.format("Property '%s' is a parameter reference in %s but not in %s", fieldName, flowAName, flowBName);
break;
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingContext.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingContext.java
index f0b1d2912e..7710e0f516 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingContext.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingContext.java
@@ -19,12 +19,12 @@ package org.apache.nifi.controller.reporting;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.stateless.engine.StatelessEngine;
import java.util.Map;
@@ -34,9 +34,9 @@ public class StatelessReportingContext extends AbstractReportingContext implemen
private final FlowManager flowManager;
public StatelessReportingContext(final StatelessEngine statelessEngine, final FlowManager flowManager,
- final Map<PropertyDescriptor, String> properties, final ReportingTask reportingTask,
+ final Map<PropertyDescriptor, String> properties, final ReportingTaskNode reportingTaskNode,
final VariableRegistry variableRegistry, final ParameterLookup parameterLookup) {
- super(reportingTask, statelessEngine.getBulletinRepository(), properties, statelessEngine.getControllerServiceProvider(), parameterLookup, variableRegistry);
+ super(reportingTaskNode, statelessEngine.getBulletinRepository(), properties, statelessEngine.getControllerServiceProvider(), parameterLookup, variableRegistry);
this.statelessEngine = statelessEngine;
this.flowManager = flowManager;
}
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java
index fdcc874420..5e218cdf31 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java
@@ -53,7 +53,7 @@ public class StatelessReportingTaskNode extends AbstractReportingTaskNode implem
@Override
public ReportingContext getReportingContext() {
- return new StatelessReportingContext(statelessEngine, flowManager, getEffectivePropertyValues(), getReportingTask(), getVariableRegistry(), getParameterLookup());
+ return new StatelessReportingContext(statelessEngine, flowManager, getEffectivePropertyValues(), this, getVariableRegistry(), getParameterLookup());
}
@Override
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SensitiveDynamicPropertiesService.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SensitiveDynamicPropertiesService.java
new file mode 100644
index 0000000000..6159d5e9f1
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SensitiveDynamicPropertiesService.java
@@ -0,0 +1,31 @@
+/*
+ * 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.cs.tests.system;
+
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+
+@SupportsSensitiveDynamicProperties
+public class SensitiveDynamicPropertiesService extends AbstractControllerService {
+
+ @Override
+ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyName) {
+ return new PropertyDescriptor.Builder().name(propertyName).addValidator(Validator.VALID).dynamic(true).build();
+ }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/SensitiveDynamicPropertiesProcessor.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/SensitiveDynamicPropertiesProcessor.java
new file mode 100644
index 0000000000..2b0e91d65a
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/SensitiveDynamicPropertiesProcessor.java
@@ -0,0 +1,74 @@
+/*
+ * 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.tests.system;
+
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.processor.AbstractProcessor;
+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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@SupportsSensitiveDynamicProperties
+public class SensitiveDynamicPropertiesProcessor extends AbstractProcessor {
+ private List<PropertyDescriptor> properties;
+ private Set<Relationship> relationships;
+
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ final List<PropertyDescriptor> properties = new ArrayList<>();
+ this.properties = Collections.unmodifiableList(properties);
+
+ final Set<Relationship> relationships = new HashSet<>();
+ this.relationships = Collections.unmodifiableSet(relationships);
+
+ }
+
+ @Override
+ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+ return new PropertyDescriptor.Builder()
+ .name(propertyDescriptorName)
+ .required(false)
+ .addValidator(Validator.VALID)
+ .dynamic(true)
+ .build();
+ }
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return relationships;
+ }
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ return properties;
+ }
+
+ @Override
+ public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+ }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/reporting/SensitiveDynamicPropertiesReportingTask.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/reporting/SensitiveDynamicPropertiesReportingTask.java
new file mode 100644
index 0000000000..03b268212e
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/reporting/SensitiveDynamicPropertiesReportingTask.java
@@ -0,0 +1,36 @@
+/*
+ * 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.reporting;
+
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+
+@SupportsSensitiveDynamicProperties
+public class SensitiveDynamicPropertiesReportingTask extends AbstractReportingTask {
+
+ @Override
+ public void onTrigger(final ReportingContext context) {
+
+ }
+
+ @Override
+ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyName) {
+ return new PropertyDescriptor.Builder().name(propertyName).addValidator(Validator.VALID).dynamic(true).build();
+ }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index 40c1ee9870..10f6db7935 100644
--- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -15,5 +15,6 @@
org.apache.nifi.cs.tests.system.EnsureControllerServiceConfigurationCorrect
org.apache.nifi.cs.tests.system.FakeControllerService1
+org.apache.nifi.cs.tests.system.SensitiveDynamicPropertiesService
org.apache.nifi.cs.tests.system.StandardCountService
org.apache.nifi.cs.tests.system.StandardSleepService
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 66fc3133bd..b0262a6dce 100644
--- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -33,6 +33,7 @@ org.apache.nifi.processors.tests.system.PassThroughRequiresInstanceClassLoading
org.apache.nifi.processors.tests.system.ReplaceWithFile
org.apache.nifi.processors.tests.system.ReverseContents
org.apache.nifi.processors.tests.system.RoundRobinFlowFiles
+org.apache.nifi.processors.tests.system.SensitiveDynamicPropertiesProcessor
org.apache.nifi.processors.tests.system.SetAttribute
org.apache.nifi.processors.tests.system.Sleep
org.apache.nifi.processors.tests.system.SplitByLine
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
index 607006fdae..4c9ba85c6c 100644
--- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
@@ -14,4 +14,5 @@
# limitations under the License.
org.apache.nifi.reporting.EnsureReportingTaskConfigurationCorrect
+org.apache.nifi.reporting.SensitiveDynamicPropertiesReportingTask
org.apache.nifi.reporting.WriteToFileReportingTask
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
index c3de536cff..2da3ac5dbe 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
@@ -533,26 +533,10 @@ public class NiFiClientUtil {
}
}
- public void waitForReportingTaskValid(final String reportingTaskId) throws NiFiClientException, IOException, InterruptedException {
+ public void waitForReportingTaskValid(final String reportingTaskId) throws NiFiClientException, IOException {
waitForReportingTaskValidationStatus(reportingTaskId, "Valid");
}
- public void waitForReportingTaskValidationStatus(final String reportingTaskId, final String expectedStatus) throws NiFiClientException, IOException, InterruptedException {
- while (true) {
- final ReportingTaskEntity entity = nifiClient.getReportingTasksClient().getReportingTask(reportingTaskId);
- final String validationStatus = entity.getComponent().getValidationStatus();
- if (expectedStatus.equalsIgnoreCase(validationStatus)) {
- return;
- }
-
- if ("Invalid".equalsIgnoreCase(validationStatus)) {
- logger.info("Reporting Task with ID {} is currently invalid due to: {}", reportingTaskId, entity.getComponent().getValidationErrors());
- }
-
- Thread.sleep(100L);
- }
- }
-
public ControllerServiceEntity updateControllerService(final ControllerServiceEntity currentEntity, final Map<String, String> properties) throws NiFiClientException, IOException {
final ControllerServiceDTO dto = new ControllerServiceDTO();
dto.setProperties(properties);
@@ -768,6 +752,48 @@ public class NiFiClientUtil {
}
}
+ public void waitForControllerServiceValidationStatus(final String controllerServiceId, final String validationStatus) throws NiFiClientException, IOException {
+ while (true) {
+ final ControllerServiceEntity controllerServiceEntity = nifiClient.getControllerServicesClient().getControllerService(controllerServiceId);
+ final String currentValidationStatus = controllerServiceEntity.getStatus().getValidationStatus();
+ if (validationStatus.equals(currentValidationStatus)) {
+ logger.info("Controller Service ID [{}] Type [{}] Validation Status [{}] matched", controllerServiceId,
+ controllerServiceEntity.getComponent().getType(), validationStatus);
+ return;
+ }
+
+ logger.info("Controller Service ID [{}] Type [{}] Validation Status [{}] waiting for [{}]: sleeping for 500 ms before retrying", controllerServiceId,
+ controllerServiceEntity.getComponent().getType(), currentValidationStatus, validationStatus);
+
+ try {
+ Thread.sleep(500L);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ public void waitForReportingTaskValidationStatus(final String reportingTaskId, final String validationStatus) throws NiFiClientException, IOException {
+ while (true) {
+ final ReportingTaskEntity reportingTaskEntity = nifiClient.getReportingTasksClient().getReportingTask(reportingTaskId);
+ final String currentValidationStatus = reportingTaskEntity.getStatus().getValidationStatus();
+ if (validationStatus.equalsIgnoreCase(currentValidationStatus)) {
+ logger.info("Reporting Task ID [{}] Type [{}] Validation Status [{}] matched", reportingTaskId,
+ reportingTaskEntity.getComponent().getType(), validationStatus);
+ return;
+ }
+
+ logger.info("Reporting Task ID [{}] Type [{}] Validation Status [{}] waiting for [{}]: sleeping for 500 ms before retrying", reportingTaskEntity,
+ reportingTaskEntity.getComponent().getType(), currentValidationStatus, validationStatus);
+
+ try {
+ Thread.sleep(500L);
+ } catch (final Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
public List<ControllerServiceEntity> getControllerServicesNotInState(final String groupId, final String desiredState, final Collection<String> serviceIds) throws NiFiClientException, IOException {
final ControllerServicesEntity servicesEntity = nifiClient.getFlowClient().getControllerServices(groupId);
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceApiValidationIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceApiValidationIT.java
index 5cc6dc07ed..8b0b164fb8 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceApiValidationIT.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceApiValidationIT.java
@@ -29,6 +29,7 @@ import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class ControllerServiceApiValidationIT extends NiFiSystemIT {
+
@Test
public void testMatchingControllerService() throws NiFiClientException, IOException {
final ControllerServiceEntity fakeServiceEntity = getClientUtil().createControllerService("FakeControllerService1");
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceDynamicPropertiesIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceDynamicPropertiesIT.java
new file mode 100644
index 0000000000..e85b6b31c0
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/ControllerServiceDynamicPropertiesIT.java
@@ -0,0 +1,100 @@
+/*
+ * 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.tests.system.controllerservice;
+
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.ControllerServiceDTO;
+import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
+import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ControllerServiceDynamicPropertiesIT extends NiFiSystemIT {
+
+ private static final String SENSITIVE_PROPERTY_NAME = "Credentials";
+
+ private static final String SENSITIVE_PROPERTY_VALUE = "Token";
+
+ private static final Set<String> SENSITIVE_DYNAMIC_PROPERTY_NAMES = Collections.singleton(SENSITIVE_PROPERTY_NAME);
+
+ @Test
+ void testGetPropertyDescriptor() throws NiFiClientException, IOException {
+ final ControllerServiceEntity controllerServiceEntity = getClientUtil().createControllerService("SensitiveDynamicPropertiesService");
+
+ final ControllerServicesClient servicesClient = getNifiClient().getControllerServicesClient();
+ final PropertyDescriptorEntity propertyDescriptorEntity = servicesClient.getPropertyDescriptor(controllerServiceEntity.getId(), SENSITIVE_PROPERTY_NAME, null);
+ final PropertyDescriptorDTO propertyDescriptor = propertyDescriptorEntity.getPropertyDescriptor();
+ assertFalse(propertyDescriptor.isSensitive());
+ assertTrue(propertyDescriptor.isDynamic());
+
+ final PropertyDescriptorEntity sensitivePropertyDescriptorEntity = servicesClient.getPropertyDescriptor(controllerServiceEntity.getId(), SENSITIVE_PROPERTY_NAME, true);
+ final PropertyDescriptorDTO sensitivePropertyDescriptor = sensitivePropertyDescriptorEntity.getPropertyDescriptor();
+ assertTrue(sensitivePropertyDescriptor.isSensitive());
+ assertTrue(sensitivePropertyDescriptor.isDynamic());
+ }
+
+ @Test
+ public void testSensitiveDynamicPropertiesNotSupported() throws NiFiClientException, IOException {
+ final ControllerServiceEntity controllerServiceEntity = getClientUtil().createControllerService("StandardCountService");
+ final ControllerServiceDTO component = controllerServiceEntity.getComponent();
+ assertFalse(component.getSupportsSensitiveDynamicProperties());
+
+ component.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+
+ getClientUtil().updateControllerService(controllerServiceEntity, Collections.singletonMap(SENSITIVE_PROPERTY_NAME, SENSITIVE_PROPERTY_VALUE));
+
+ getClientUtil().waitForControllerServiceValidationStatus(controllerServiceEntity.getId(), ControllerServiceDTO.INVALID);
+ }
+
+ @Test
+ public void testSensitiveDynamicPropertiesSupportedConfigured() throws NiFiClientException, IOException {
+ final ControllerServiceEntity controllerServiceEntity = getClientUtil().createControllerService("SensitiveDynamicPropertiesService");
+ final ControllerServiceDTO component = controllerServiceEntity.getComponent();
+ assertTrue(component.getSupportsSensitiveDynamicProperties());
+
+ component.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+ component.setProperties(Collections.singletonMap(SENSITIVE_PROPERTY_NAME, SENSITIVE_PROPERTY_VALUE));
+
+ getNifiClient().getControllerServicesClient().updateControllerService(controllerServiceEntity);
+
+ final ControllerServiceEntity updatedControllerServiceEntity = getNifiClient().getControllerServicesClient().getControllerService(controllerServiceEntity.getId());
+ final ControllerServiceDTO updatedComponent = updatedControllerServiceEntity.getComponent();
+
+ final Map<String, String> properties = updatedComponent.getProperties();
+ assertNotSame(SENSITIVE_PROPERTY_VALUE, properties.get(SENSITIVE_PROPERTY_NAME));
+
+ final Map<String, PropertyDescriptorDTO> descriptors = updatedComponent.getDescriptors();
+ final PropertyDescriptorDTO descriptor = descriptors.get(SENSITIVE_PROPERTY_NAME);
+ assertNotNull(descriptor);
+ assertTrue(descriptor.isSensitive());
+ assertTrue(descriptor.isDynamic());
+
+ getClientUtil().waitForControllerServiceValidationStatus(controllerServiceEntity.getId(), ControllerServiceDTO.VALID);
+ }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/processor/ProcessorDynamicPropertiesIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/processor/ProcessorDynamicPropertiesIT.java
new file mode 100644
index 0000000000..9a4d0c9e3a
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/processor/ProcessorDynamicPropertiesIT.java
@@ -0,0 +1,102 @@
+/*
+ * 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.tests.system.processor;
+
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
+import org.apache.nifi.web.api.dto.ProcessorDTO;
+import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ProcessorDynamicPropertiesIT extends NiFiSystemIT {
+ private static final String SENSITIVE_PROPERTY_NAME = "Credentials";
+
+ private static final String SENSITIVE_PROPERTY_VALUE = "Token";
+
+ private static final Set<String> SENSITIVE_DYNAMIC_PROPERTY_NAMES = Collections.singleton(SENSITIVE_PROPERTY_NAME);
+
+ @Test
+ void testGetPropertyDescriptor() throws NiFiClientException, IOException {
+ final ProcessorEntity processorEntity = getClientUtil().createProcessor("SensitiveDynamicPropertiesProcessor");
+
+ final PropertyDescriptorEntity propertyDescriptorEntity = getNifiClient().getProcessorClient().getPropertyDescriptor(processorEntity.getId(), SENSITIVE_PROPERTY_NAME, null);
+ final PropertyDescriptorDTO propertyDescriptor = propertyDescriptorEntity.getPropertyDescriptor();
+ assertFalse(propertyDescriptor.isSensitive());
+ assertTrue(propertyDescriptor.isDynamic());
+
+ final PropertyDescriptorEntity sensitivePropertyDescriptorEntity = getNifiClient().getProcessorClient().getPropertyDescriptor(processorEntity.getId(), SENSITIVE_PROPERTY_NAME, true);
+ final PropertyDescriptorDTO sensitivePropertyDescriptor = sensitivePropertyDescriptorEntity.getPropertyDescriptor();
+ assertTrue(sensitivePropertyDescriptor.isSensitive());
+ assertTrue(sensitivePropertyDescriptor.isDynamic());
+ }
+
+ @Test
+ void testSensitiveDynamicPropertiesNotSupported() throws NiFiClientException, IOException, InterruptedException {
+ final ProcessorEntity processorEntity = getClientUtil().createProcessor("FakeDynamicPropertiesProcessor");
+ final ProcessorDTO component = processorEntity.getComponent();
+ assertFalse(component.getSupportsSensitiveDynamicProperties());
+
+ final ProcessorConfigDTO config = component.getConfig();
+ config.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+ config.setProperties(Collections.singletonMap(SENSITIVE_PROPERTY_NAME, SENSITIVE_PROPERTY_VALUE));
+
+ getClientUtil().updateProcessorConfig(processorEntity, config);
+
+ getClientUtil().waitForInvalidProcessor(processorEntity.getId());
+ }
+
+ @Test
+ void testSensitiveDynamicPropertiesSupportedConfigured() throws NiFiClientException, IOException, InterruptedException {
+ final ProcessorEntity processorEntity = getClientUtil().createProcessor("SensitiveDynamicPropertiesProcessor");
+ final ProcessorDTO component = processorEntity.getComponent();
+ assertTrue(component.getSupportsSensitiveDynamicProperties());
+
+ final ProcessorConfigDTO config = component.getConfig();
+ config.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+ config.setProperties(Collections.singletonMap(SENSITIVE_PROPERTY_NAME, SENSITIVE_PROPERTY_VALUE));
+
+ getClientUtil().updateProcessorConfig(processorEntity, config);
+
+ getClientUtil().waitForValidProcessor(processorEntity.getId());
+
+ final ProcessorEntity updatedProcessorEntity = getNifiClient().getProcessorClient().getProcessor(processorEntity.getId());
+ final ProcessorDTO updatedComponent = updatedProcessorEntity.getComponent();
+ final ProcessorConfigDTO updatedConfig = updatedComponent.getConfig();
+
+ final Map<String, String> properties = updatedConfig.getProperties();
+ assertNotSame(SENSITIVE_PROPERTY_VALUE, properties.get(SENSITIVE_PROPERTY_NAME));
+
+ final Map<String, PropertyDescriptorDTO> descriptors = updatedConfig.getDescriptors();
+ final PropertyDescriptorDTO descriptor = descriptors.get(SENSITIVE_PROPERTY_NAME);
+ assertNotNull(descriptor);
+ assertTrue(descriptor.isSensitive());
+ assertTrue(descriptor.isDynamic());
+ }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/reportingtask/ReportingTaskIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/reportingtask/ReportingTaskIT.java
index c9e1fdf69f..7c60811ff9 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/reportingtask/ReportingTaskIT.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/reportingtask/ReportingTaskIT.java
@@ -19,7 +19,10 @@ package org.apache.nifi.tests.system.reportingtask;
import org.apache.nifi.tests.system.NiFiSystemIT;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
+import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.junit.jupiter.api.Test;
@@ -27,11 +30,25 @@ import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
+import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
public class ReportingTaskIT extends NiFiSystemIT {
+ private static final String SENSITIVE_PROPERTY_NAME = "Credentials";
+
+ private static final String SENSITIVE_PROPERTY_VALUE = "Token";
+
+ private static final Set<String> SENSITIVE_DYNAMIC_PROPERTY_NAMES = Collections.singleton(SENSITIVE_PROPERTY_NAME);
+
+
@Test
public void testReportingTaskDependingOnControllerService() throws NiFiClientException, IOException, InterruptedException {
// Create a Count controller service and configure a property on the service.
@@ -71,6 +88,60 @@ public class ReportingTaskIT extends NiFiSystemIT {
waitFor(() -> getCount(reportingTaskFile) >= 1_000_005);
}
+ @Test
+ public void testGetPropertyDescriptor() throws NiFiClientException, IOException {
+ final ReportingTaskEntity reportingTaskEntity = getClientUtil().createReportingTask("SensitiveDynamicPropertiesReportingTask");
+
+ final PropertyDescriptorEntity propertyDescriptorEntity = getNifiClient().getReportingTasksClient().getPropertyDescriptor(reportingTaskEntity.getId(), SENSITIVE_PROPERTY_NAME, null);
+ final PropertyDescriptorDTO propertyDescriptor = propertyDescriptorEntity.getPropertyDescriptor();
+ assertFalse(propertyDescriptor.isSensitive());
+ assertTrue(propertyDescriptor.isDynamic());
+
+ final PropertyDescriptorEntity sensitivePropertyDescriptorEntity = getNifiClient().getReportingTasksClient().getPropertyDescriptor(reportingTaskEntity.getId(), SENSITIVE_PROPERTY_NAME, true);
+ final PropertyDescriptorDTO sensitivePropertyDescriptor = sensitivePropertyDescriptorEntity.getPropertyDescriptor();
+ assertTrue(sensitivePropertyDescriptor.isSensitive());
+ assertTrue(sensitivePropertyDescriptor.isDynamic());
+ }
+
+ @Test
+ public void testSensitiveDynamicPropertiesNotSupported() throws NiFiClientException, IOException {
+ final ReportingTaskEntity reportingTaskEntity = getClientUtil().createReportingTask("WriteToFileReportingTask");
+ final ReportingTaskDTO component = reportingTaskEntity.getComponent();
+ assertFalse(component.getSupportsSensitiveDynamicProperties());
+
+ component.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+
+ getNifiClient().getReportingTasksClient().updateReportingTask(reportingTaskEntity);
+
+ getClientUtil().waitForReportingTaskValidationStatus(reportingTaskEntity.getId(), ReportingTaskDTO.INVALID);
+ }
+
+ @Test
+ public void testSensitiveDynamicPropertiesSupportedConfigured() throws NiFiClientException, IOException {
+ final ReportingTaskEntity reportingTaskEntity = getClientUtil().createReportingTask("SensitiveDynamicPropertiesReportingTask");
+ final ReportingTaskDTO component = reportingTaskEntity.getComponent();
+ assertTrue(component.getSupportsSensitiveDynamicProperties());
+
+ component.setSensitiveDynamicPropertyNames(SENSITIVE_DYNAMIC_PROPERTY_NAMES);
+ component.setProperties(Collections.singletonMap(SENSITIVE_PROPERTY_NAME, SENSITIVE_PROPERTY_VALUE));
+
+ getNifiClient().getReportingTasksClient().updateReportingTask(reportingTaskEntity);
+
+ final ReportingTaskEntity updatedReportingTaskEntity = getNifiClient().getReportingTasksClient().getReportingTask(reportingTaskEntity.getId());
+ final ReportingTaskDTO updatedComponent = updatedReportingTaskEntity.getComponent();
+
+ final Map<String, String> properties = updatedComponent.getProperties();
+ assertNotSame(SENSITIVE_PROPERTY_VALUE, properties.get(SENSITIVE_PROPERTY_NAME));
+
+ final Map<String, PropertyDescriptorDTO> descriptors = updatedComponent.getDescriptors();
+ final PropertyDescriptorDTO descriptor = descriptors.get(SENSITIVE_PROPERTY_NAME);
+ assertNotNull(descriptor);
+ assertTrue(descriptor.isSensitive());
+ assertTrue(descriptor.isDynamic());
+
+ getClientUtil().waitForReportingTaskValid(reportingTaskEntity.getId());
+ }
+
private long getCount(final File file) throws IOException {
final byte[] bytes = Files.readAllBytes(file.toPath());
final String contents = new String(bytes, StandardCharsets.UTF_8);
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
index f9ad54c1e7..1c4128872d 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
@@ -19,6 +19,7 @@ package org.apache.nifi.toolkit.cli.impl.client.nifi;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
@@ -49,4 +50,5 @@ public interface ControllerServicesClient {
VerifyConfigRequestEntity deleteConfigVerificationRequest(String serviceId, String verificationRequestId) throws NiFiClientException, IOException;
+ PropertyDescriptorEntity getPropertyDescriptor(String serviceId, String propertyName, Boolean sensitive) throws NiFiClientException, IOException;
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java
index 75382704d6..1b355fbe8c 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java
@@ -17,6 +17,7 @@
package org.apache.nifi.toolkit.cli.impl.client.nifi;
import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
import java.io.IOException;
@@ -54,6 +55,8 @@ public interface ProcessorClient {
VerifyConfigRequestEntity deleteConfigVerificationRequest(String processorId, String verificationRequestId) throws NiFiClientException, IOException;
+ PropertyDescriptorEntity getPropertyDescriptor(String processorId, String propertyName, Boolean sensitive) throws NiFiClientException, IOException;
+
/**
* Indicates that mutable requests should indicate that the client has acknowledged that the node is disconnected.
*/
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ReportingTasksClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ReportingTasksClient.java
index 38e9ebaa60..478d2f2b00 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ReportingTasksClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ReportingTasksClient.java
@@ -17,6 +17,7 @@
*/
package org.apache.nifi.toolkit.cli.impl.client.nifi;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
@@ -42,4 +43,5 @@ public interface ReportingTasksClient {
ReportingTaskEntity deleteReportingTask(ReportingTaskEntity reportingTask) throws NiFiClientException, IOException;
+ PropertyDescriptorEntity getPropertyDescriptor(String taskId, String propertyName, Boolean sensitive) throws NiFiClientException, IOException;
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
index 7317444c4c..fb6722d3cd 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
@@ -24,6 +24,7 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
@@ -31,6 +32,7 @@ import javax.ws.rs.client.Entity;
import javax.ws.rs.client.WebTarget;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
+import java.util.Objects;
/**
* Jersey implementation of ControllerServicersClient.
@@ -240,4 +242,20 @@ public class JerseyControllerServicesClient extends AbstractJerseyClient impleme
return getRequestBuilder(target).delete(VerifyConfigRequestEntity.class);
});
}
+
+ @Override
+ public PropertyDescriptorEntity getPropertyDescriptor(final String serviceId, final String propertyName, final Boolean sensitive) throws NiFiClientException, IOException {
+ Objects.requireNonNull(serviceId, "Service ID required");
+ Objects.requireNonNull(propertyName, "Property Name required");
+
+ return executeAction("Error retrieving Property Descriptor", () -> {
+ final WebTarget target = controllerServicesTarget
+ .path("{id}/descriptors")
+ .resolveTemplate("id", serviceId)
+ .queryParam("propertyName", propertyName)
+ .queryParam("sensitive", sensitive);
+
+ return getRequestBuilder(target).get(PropertyDescriptorEntity.class);
+ });
+ }
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java
index c0c52a3d2e..3de97619b7 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java
@@ -23,12 +23,14 @@ import org.apache.nifi.toolkit.cli.impl.client.nifi.RequestConfig;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.ProcessorRunStatusEntity;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
import javax.ws.rs.client.Entity;
import javax.ws.rs.client.WebTarget;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
+import java.util.Objects;
public class JerseyProcessorClient extends AbstractJerseyClient implements ProcessorClient {
private volatile WebTarget processGroupTarget;
@@ -257,4 +259,20 @@ public class JerseyProcessorClient extends AbstractJerseyClient implements Proce
return getRequestBuilder(target).delete(VerifyConfigRequestEntity.class);
});
}
+
+ @Override
+ public PropertyDescriptorEntity getPropertyDescriptor(final String processorId, final String propertyName, final Boolean sensitive) throws NiFiClientException, IOException {
+ Objects.requireNonNull(processorId, "Processor ID required");
+ Objects.requireNonNull(propertyName, "Property Name required");
+
+ return executeAction("Error retrieving Property Descriptor", () -> {
+ final WebTarget target = processorTarget
+ .path("/descriptors")
+ .resolveTemplate("id", processorId)
+ .queryParam("propertyName", propertyName)
+ .queryParam("sensitive", sensitive);
+
+ return getRequestBuilder(target).get(PropertyDescriptorEntity.class);
+ });
+ }
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyReportingTasksClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyReportingTasksClient.java
index 82a11b4a87..1094d352f1 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyReportingTasksClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyReportingTasksClient.java
@@ -22,6 +22,7 @@ import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
import org.apache.nifi.toolkit.cli.impl.client.nifi.RequestConfig;
import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity;
@@ -30,6 +31,7 @@ import javax.ws.rs.client.Entity;
import javax.ws.rs.client.WebTarget;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
+import java.util.Objects;
/**
* Jersey implementation of ReportingTasksClient.
@@ -183,4 +185,19 @@ public class JerseyReportingTasksClient extends AbstractJerseyClient implements
});
}
+ @Override
+ public PropertyDescriptorEntity getPropertyDescriptor(final String componentId, final String propertyName, final Boolean sensitive) throws NiFiClientException, IOException {
+ Objects.requireNonNull(componentId, "Component ID required");
+ Objects.requireNonNull(propertyName, "Property Name required");
+
+ return executeAction("Error retrieving Property Descriptor", () -> {
+ final WebTarget target = reportingTasksTarget
+ .path("{id}/descriptors")
+ .resolveTemplate("id", componentId)
+ .queryParam("propertyName", propertyName)
+ .queryParam("sensitive", sensitive);
+
+ return getRequestBuilder(target).get(PropertyDescriptorEntity.class);
+ });
+ }
}