You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2020/11/03 20:39:10 UTC

[nifi] branch main updated (d8d9aa9 -> d773521)

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

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


    from d8d9aa9  Added regex support for attribute header selection on HandleHTTPResponse
     new f7f336a  NIFI-1121: Added API changes for having one Property depend on another
     new 4bd9d7b  NIFI-1121 Show and hide properties that depend on another property.
     new e2e901b  NIFI-1121: Added property dependencies to MergeContent
     new 535cab3  NIFI-1121: Added an additional check for hidden properties to account for transitive dependent properties. - Added a 'dependent' attribute to determine whether or not to save dependent property values
     new 4b9014b  NIFI-1121: Updated backend to perform appropriate validation. Added tests. Updated documentation writer. Updated dev guide to explain how PropertyDescriptor.Builder#dependsOn affects validation. Updated JavaDocs for PropertyDescriptor.Builder#dependsOn
     new 42c2cda  NIFI-1121 Fixed a dependent value check error.
     new d773521  NIFI-1121 Fix Schema Name and Schema Branch properties

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../components/AbstractConfigurableComponent.java  |  74 ++++++++++--
 .../apache/nifi/components/PropertyDependency.java |  98 ++++++++++++++++
 .../apache/nifi/components/PropertyDescriptor.java |  86 +++++++++++++-
 .../apache/nifi/components/ValidationContext.java  |  23 +++-
 .../documentation/xml/XmlDocumentationWriter.java  |  30 +++++
 .../NotificationValidationContext.java             |   6 +
 nifi-docs/src/main/asciidoc/developer-guide.adoc   |  37 +++++-
 nifi-mock/pom.xml                                  |   5 +
 .../org/apache/nifi/util/MockProcessContext.java   |  50 +++++----
 .../org/apache/nifi/util/MockPropertyValue.java    |  28 ++---
 .../apache/nifi/util/MockValidationContext.java    | 124 +++++++++++++++++++--
 .../nifi/util/StandardProcessorTestRunner.java     |  15 ++-
 .../main/java/org/apache/nifi/util/TestRunner.java |  29 ++---
 .../apache/nifi/processor/util/bin/BinFiles.java   |   3 +-
 .../nifi/schema/access/SchemaAccessUtils.java      |  21 ++--
 .../SchemaRegistryRecordSetWriter.java             |  28 ++---
 ...RegistryDTO.java => PropertyDependencyDTO.java} |  29 +++--
 .../nifi/web/api/dto/PropertyDescriptorDTO.java    |  11 ++
 .../html/HtmlDocumentationWriter.java              |  47 ++++++++
 .../pom.xml                                        |  13 ++-
 .../org/apache/nifi/AbstractValidationContext.java | 110 ++++++++++++++++++
 .../nifi/controller/AbstractComponentNode.java     |   9 ++
 .../nifi-framework/nifi-framework-core/pom.xml     |   5 +
 .../nifi/processor/StandardValidationContext.java  |   6 +-
 .../nifi/controller/TestStandardProcessorNode.java |   7 ++
 .../nifi-framework/nifi-stateless/pom.xml          |   5 +
 .../stateless/core/StatelessProcessContext.java    |   4 +
 .../stateless/core/StatelessValidationContext.java |  20 +++-
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  15 +++
 .../jquery/propertytable/jquery.propertytable.js   | 118 +++++++++++++++++++-
 .../nifi-framework-bundle/nifi-framework/pom.xml   |   1 +
 .../nifi/script/impl/ValidationContextAdapter.java |   6 +
 .../nifi/processors/standard/CompressContent.java  | 102 ++++++++---------
 .../nifi/processors/standard/MergeContent.java     |  66 ++++++-----
 .../nifi/processors/standard/TestMergeContent.java |  36 +++---
 .../processors/standard/TestValidateRecord.java    |   1 -
 .../org/apache/nifi/json/JsonRecordSetWriter.java  |   1 +
 .../java/org/apache/nifi/json/JsonTreeReader.java  |   9 +-
 .../jetty/ControllerServiceTestContext.java        |   4 +
 .../tests/system/DependOnProperties.java           | 117 +++++++++++++++++++
 .../services/org.apache.nifi.processor.Processor   |   1 +
 .../validation/DependentPropertyValidationIT.java  |  99 ++++++++++++++++
 42 files changed, 1272 insertions(+), 227 deletions(-)
 create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
 copy nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/{VariableRegistryDTO.java => PropertyDependencyDTO.java} (55%)
 copy nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-standard-prioritizers => nifi-framework-components}/pom.xml (82%)
 create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/AbstractValidationContext.java
 create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
 create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/validation/DependentPropertyValidationIT.java


[nifi] 03/07: NIFI-1121: Added property dependencies to MergeContent

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e2e901b6b97c8099038230b430905bd5c85bf6ae
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Aug 14 16:42:47 2020 -0400

    NIFI-1121: Added property dependencies to MergeContent
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../nifi/processors/standard/MergeContent.java     | 60 +++++++++++++---------
 1 file changed, 37 insertions(+), 23 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 5d4506c..215822e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -166,19 +166,6 @@ public class MergeContent extends BinFiles {
     public static final AllowableValue METADATA_STRATEGY_DO_NOT_MERGE = new AllowableValue("Do Not Merge Uncommon Metadata", "Do Not Merge Uncommon Metadata",
             "For any input format that supports metadata (Avro, e.g.), any FlowFile whose metadata values do not match those of the first FlowFile in the bin will not be merged.");
 
-    public static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
-            .required(true)
-            .name("mergecontent-metadata-strategy")
-            .displayName("Metadata Strategy")
-            .description("For FlowFiles whose input format supports metadata (Avro, e.g.), this property determines which metadata should be added to the bundle. "
-                    + "If 'Use First Metadata' is selected, the metadata keys/values from the first FlowFile to be bundled will be used. If 'Keep Only Common Metadata' is selected, "
-                    + "only the metadata that exists on all FlowFiles in the bundle, with the same value, will be preserved. If 'Ignore Metadata' is selected, no metadata is transferred to "
-                    + "the outgoing bundled FlowFile. If 'Do Not Merge Uncommon Metadata' is selected, any FlowFile whose metadata values do not match those of the first bundled FlowFile "
-                    + "will not be merged.")
-            .allowableValues(METADATA_STRATEGY_USE_FIRST, METADATA_STRATEGY_ALL_COMMON, METADATA_STRATEGY_DO_NOT_MERGE, METADATA_STRATEGY_IGNORE)
-            .defaultValue(METADATA_STRATEGY_DO_NOT_MERGE.getValue())
-            .build();
-
     public static final AllowableValue MERGE_STRATEGY_BIN_PACK = new AllowableValue(
             "Bin-Packing Algorithm",
             "Bin-Packing Algorithm",
@@ -197,6 +184,8 @@ public class MergeContent extends BinFiles {
             "Filename", "Filename", "The values of Header, Footer, and Demarcator will be retrieved from the contents of a file");
     public static final AllowableValue DELIMITER_STRATEGY_TEXT = new AllowableValue(
             "Text", "Text", "The values of Header, Footer, and Demarcator will be specified as property values");
+    public static final AllowableValue DELIMITER_STRATEGY_NONE = new AllowableValue(
+        "Do Not Use Delimiters", "Do Not Use Delimiters", "No Header, Footer, or Demarcator will be used");
 
     public static final String MERGE_FORMAT_TAR_VALUE = "TAR";
     public static final String MERGE_FORMAT_ZIP_VALUE = "ZIP";
@@ -263,6 +252,20 @@ public class MergeContent extends BinFiles {
             .defaultValue(MERGE_FORMAT_CONCAT.getValue())
             .build();
 
+    public static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+        .required(true)
+        .name("mergecontent-metadata-strategy")
+        .displayName("Metadata Strategy")
+        .description("For FlowFiles whose input format supports metadata (Avro, e.g.), this property determines which metadata should be added to the bundle. "
+            + "If 'Use First Metadata' is selected, the metadata keys/values from the first FlowFile to be bundled will be used. If 'Keep Only Common Metadata' is selected, "
+            + "only the metadata that exists on all FlowFiles in the bundle, with the same value, will be preserved. If 'Ignore Metadata' is selected, no metadata is transferred to "
+            + "the outgoing bundled FlowFile. If 'Do Not Merge Uncommon Metadata' is selected, any FlowFile whose metadata values do not match those of the first bundled FlowFile "
+            + "will not be merged.")
+        .allowableValues(METADATA_STRATEGY_USE_FIRST, METADATA_STRATEGY_ALL_COMMON, METADATA_STRATEGY_DO_NOT_MERGE, METADATA_STRATEGY_IGNORE)
+        .defaultValue(METADATA_STRATEGY_DO_NOT_MERGE.getValue())
+        .dependsOn(MERGE_FORMAT, MERGE_FORMAT_AVRO)
+        .build();
+
     public static final PropertyDescriptor CORRELATION_ATTRIBUTE_NAME = new PropertyDescriptor.Builder()
             .name("Correlation Attribute Name")
             .description("If specified, like FlowFiles will be binned together, where 'like FlowFiles' means FlowFiles that have the same value for "
@@ -271,6 +274,7 @@ public class MergeContent extends BinFiles {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
             .defaultValue(null)
+            .dependsOn(MERGE_STRATEGY, MERGE_STRATEGY_BIN_PACK)
             .build();
 
     public static final PropertyDescriptor DELIMITER_STRATEGY = new PropertyDescriptor.Builder()
@@ -278,8 +282,9 @@ public class MergeContent extends BinFiles {
             .name("Delimiter Strategy")
             .description("Determines if Header, Footer, and Demarcator should point to files containing the respective content, or if "
                     + "the values of the properties should be used as the content.")
-            .allowableValues(DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
-            .defaultValue(DELIMITER_STRATEGY_FILENAME.getValue())
+            .allowableValues(DELIMITER_STRATEGY_NONE, DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
+            .defaultValue(DELIMITER_STRATEGY_NONE.getValue())
+            .dependsOn(MERGE_STRATEGY, MERGE_STRATEGY_BIN_PACK)
             .build();
     public static final PropertyDescriptor HEADER = new PropertyDescriptor.Builder()
             .name("Header File")
@@ -289,6 +294,7 @@ public class MergeContent extends BinFiles {
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(DELIMITER_STRATEGY, DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
             .build();
     public static final PropertyDescriptor FOOTER = new PropertyDescriptor.Builder()
             .name("Footer File")
@@ -298,6 +304,7 @@ public class MergeContent extends BinFiles {
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(DELIMITER_STRATEGY, DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
             .build();
     public static final PropertyDescriptor DEMARCATOR = new PropertyDescriptor.Builder()
             .name("Demarcator File")
@@ -307,6 +314,7 @@ public class MergeContent extends BinFiles {
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(DELIMITER_STRATEGY, DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
             .build();
     public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
             .name("Compression Level")
@@ -315,24 +323,25 @@ public class MergeContent extends BinFiles {
             .required(true)
             .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
             .defaultValue("1")
+            .dependsOn(MERGE_FORMAT, MERGE_FORMAT_ZIP)
             .build();
     public static final PropertyDescriptor KEEP_PATH = new PropertyDescriptor.Builder()
             .name("Keep Path")
-            .description("If using the Zip or Tar Merge Format, specifies whether or not the FlowFiles' paths should be included in their entry "
-                    + "names; if using other merge strategy, this value is ignored")
+            .description("If using the Zip or Tar Merge Format, specifies whether or not the FlowFiles' paths should be included in their entry names.")
             .required(true)
             .allowableValues("true", "false")
             .defaultValue("false")
+            .dependsOn(MERGE_FORMAT, MERGE_FORMAT_TAR, MERGE_FORMAT_ZIP)
             .build();
     public static final PropertyDescriptor TAR_MODIFIED_TIME = new PropertyDescriptor.Builder()
             .name("Tar Modified Time")
             .description("If using the Tar Merge Format, specifies if the Tar entry should store the modified timestamp either by expression "
-                    + "(e.g. ${file.lastModifiedTime} or static value, both of which must match the ISO8601 format 'yyyy-MM-dd'T'HH:mm:ssZ'; if using "
-                    + "other merge strategy or left blank, this value is ignored")
+                    + "(e.g. ${file.lastModifiedTime} or static value, both of which must match the ISO8601 format 'yyyy-MM-dd'T'HH:mm:ssZ'.")
             .required(false)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("${file.lastModifiedTime}")
+            .dependsOn(MERGE_FORMAT, MERGE_FORMAT_TAR)
             .build();
 
     public static final Relationship REL_MERGED = new Relationship.Builder().name("merged").description("The FlowFile containing the merged content").build();
@@ -356,10 +365,10 @@ public class MergeContent extends BinFiles {
         descriptors.add(AttributeStrategyUtil.ATTRIBUTE_STRATEGY);
         descriptors.add(CORRELATION_ATTRIBUTE_NAME);
         descriptors.add(METADATA_STRATEGY);
-        descriptors.add(MIN_ENTRIES);
-        descriptors.add(MAX_ENTRIES);
-        descriptors.add(MIN_SIZE);
-        descriptors.add(MAX_SIZE);
+        descriptors.add(addBinPackingDependency(MIN_ENTRIES));
+        descriptors.add(addBinPackingDependency(MAX_ENTRIES));
+        descriptors.add(addBinPackingDependency(MIN_SIZE));
+        descriptors.add(addBinPackingDependency(MAX_SIZE));
         descriptors.add(MAX_BIN_AGE);
         descriptors.add(MAX_BIN_COUNT);
         descriptors.add(DELIMITER_STRATEGY);
@@ -372,6 +381,11 @@ public class MergeContent extends BinFiles {
         return descriptors;
     }
 
+    // Convenience method to make creation of property descriptors cleaner
+    private PropertyDescriptor addBinPackingDependency(final PropertyDescriptor original) {
+        return new PropertyDescriptor.Builder().fromPropertyDescriptor(original).dependsOn(MERGE_STRATEGY, MERGE_STRATEGY_BIN_PACK).build();
+    }
+
     @Override
     protected Collection<ValidationResult> additionalCustomValidation(ValidationContext context) {
         final Collection<ValidationResult> results = new ArrayList<>();


[nifi] 01/07: NIFI-1121: Added API changes for having one Property depend on another

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f7f336a4b0ad121e127a7c8c539bbb110ca53b40
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Mon Jul 6 17:19:28 2020 -0400

    NIFI-1121: Added API changes for having one Property depend on another
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../apache/nifi/components/PropertyDependency.java |  86 +++++++++++++++++
 .../apache/nifi/components/PropertyDescriptor.java |  90 +++++++++++++++++-
 .../nifi/schema/access/SchemaAccessUtils.java      |  21 +++--
 .../nifi/web/api/dto/PropertyDependencyDTO.java    |  47 ++++++++++
 .../nifi/web/api/dto/PropertyDescriptorDTO.java    |  11 +++
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |  15 +++
 .../nifi/processors/standard/CompressContent.java  | 102 ++++++++++-----------
 7 files changed, 309 insertions(+), 63 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
new file mode 100644
index 0000000..5c0f9b5
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
@@ -0,0 +1,86 @@
+/*
+ * 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.components;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+public class PropertyDependency {
+    private final String propertyName;
+    private final Set<String> dependentValues;
+
+    /**
+     * Creates a dependency that is satisfied if any value is set for the property with the given name
+     * @param propertyName the name of the property that is depended upon
+     */
+    public PropertyDependency(final String propertyName) {
+        this.propertyName = Objects.requireNonNull(propertyName);
+        this.dependentValues = null;
+    }
+
+    /**
+     * Creates a dependency that is satisfied only if the property with the given name has a value that is in the given set of dependent values
+     * @param propertyName the name of the property that is depended upon
+     * @param dependentValues the values that satisfy the dependency
+     */
+    public PropertyDependency(final String propertyName, final Set<String> dependentValues) {
+        this.propertyName = Objects.requireNonNull(propertyName);
+        this.dependentValues = Collections.unmodifiableSet(new HashSet<>(Objects.requireNonNull(dependentValues)));
+    }
+
+    /**
+     * @return the name of the property that is depended upon
+     */
+    public String getPropertyName() {
+        return propertyName;
+    }
+
+    /**
+     * @return the Set of values that satisfy the dependency
+     */
+    public Set<String> getDependentValues() {
+        return dependentValues;
+    }
+
+    @Override
+    public String toString() {
+        return "PropertyDependency[propertyName=" + propertyName + ", dependentValues=" + dependentValues + "]";
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final PropertyDependency that = (PropertyDependency) o;
+        return Objects.equals(getPropertyName(), that.getPropertyName()) &&
+            Objects.equals(getDependentValues(), that.getDependentValues());
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(getPropertyName(), getDependentValues());
+    }
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
index 0e23510..012a702 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
@@ -20,6 +20,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import org.apache.nifi.controller.ControllerService;
@@ -104,12 +105,18 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
      */
     private final List<Validator> validators;
 
+    /**
+     * The list of dependencies that this property has on other properties
+     */
+    private final Set<PropertyDependency> dependencies;
+
+
     protected PropertyDescriptor(final Builder builder) {
         this.displayName = builder.displayName == null ? builder.name : builder.displayName;
         this.name = builder.name;
         this.description = builder.description;
         this.defaultValue = builder.defaultValue;
-        this.allowableValues = builder.allowableValues;
+        this.allowableValues = builder.allowableValues == null ? null : Collections.unmodifiableList(new ArrayList<>(builder.allowableValues));
         this.required = builder.required;
         this.sensitive = builder.sensitive;
         this.dynamic = builder.dynamic;
@@ -117,7 +124,8 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
         this.expressionLanguageSupported = builder.expressionLanguageSupported;
         this.expressionLanguageScope = builder.expressionLanguageScope;
         this.controllerServiceDefinition = builder.controllerServiceDefinition;
-        this.validators = new ArrayList<>(builder.validators);
+        this.validators = Collections.unmodifiableList(new ArrayList<>(builder.validators));
+        this.dependencies = builder.dependencies == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(builder.dependencies));
     }
 
     @Override
@@ -185,6 +193,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
         private String description = "";
         private String defaultValue = null;
         private List<AllowableValue> allowableValues = null;
+        private Set<PropertyDependency> dependencies = null;
         private boolean required = false;
         private boolean sensitive = false;
 
@@ -211,6 +220,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
             this.expressionLanguageScope = specDescriptor.expressionLanguageScope;
             this.controllerServiceDefinition = specDescriptor.getControllerServiceDefinition();
             this.validators = new ArrayList<>(specDescriptor.validators);
+            this.dependencies = new HashSet<>(specDescriptor.dependencies);
             return this;
         }
 
@@ -444,6 +454,74 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
         }
 
         /**
+         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a non-null value.
+         * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
+         * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
+         *
+         * @param property the property that must be set in order for this property to become relevant
+         * @param dependentValues the possible values for the given property for which this Property is relevant
+         * @return the builder
+         */
+        public Builder dependsOn(final PropertyDescriptor property, final AllowableValue... dependentValues) {
+            if (dependencies == null) {
+                dependencies = new HashSet<>();
+            }
+
+            if (dependentValues.length == 0) {
+                dependencies.add(new PropertyDependency(property.getName()));
+            } else {
+                final Set<String> dependentValueSet = new HashSet<>();
+                for (final AllowableValue value : dependentValues) {
+                    dependentValueSet.add(value.getValue());
+                }
+
+                dependencies.add(new PropertyDependency(property.getName(), dependentValueSet));
+            }
+
+            return this;
+        }
+
+
+        /**
+         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a non-null value.
+         * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
+         * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
+         *
+         * @param property the property that must be set in order for this property to become relevant
+         * @param dependentValues the possible values for the given property for which this Property is relevant
+         * @return the builder
+         */
+        public Builder dependsOn(final PropertyDescriptor property, final String... dependentValues) {
+            return dependsOn(property.getName(), dependentValues);
+        }
+
+
+        /**
+         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a non-null value.
+         * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
+         * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
+         *
+         * @param propertyName the property that must be set in order for this property to become relevant
+         * @param dependentValues the possible values for the given property for which this Property is relevant
+         * @return the builder
+         */
+        public Builder dependsOn(final String propertyName, final String... dependentValues) {
+            if (dependencies == null) {
+                dependencies = new HashSet<>();
+            }
+
+            if (dependentValues.length == 0) {
+                dependencies.add(new PropertyDependency(propertyName));
+            } else {
+                final Set<String> dependentValueSet = new HashSet<>(Arrays.asList(dependentValues));
+                dependencies.add(new PropertyDependency(propertyName, dependentValueSet));
+            }
+
+            return this;
+        }
+
+
+        /**
          * @return a PropertyDescriptor as configured
          *
          * @throws IllegalStateException if allowable values are configured but
@@ -507,11 +585,15 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
     }
 
     public List<Validator> getValidators() {
-        return Collections.unmodifiableList(validators);
+        return validators;
     }
 
     public List<AllowableValue> getAllowableValues() {
-        return allowableValues == null ? null : Collections.unmodifiableList(allowableValues);
+        return allowableValues;
+    }
+
+    public Set<PropertyDependency> getDependencies() {
+        return dependencies;
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
index 7921dff..31612e6 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
@@ -52,14 +52,6 @@ public class SchemaAccessUtils {
             + "This is based on version 3.2.x of the Confluent Schema Registry.");
     public static final AllowableValue INFER_SCHEMA = new AllowableValue("infer", "Infer from Result");
 
-    public static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
-            .name("schema-registry")
-            .displayName("Schema Registry")
-            .description("Specifies the Controller Service to use for the Schema Registry")
-            .identifiesControllerService(SchemaRegistry.class)
-            .required(false)
-            .build();
-
     public static final PropertyDescriptor SCHEMA_ACCESS_STRATEGY = new PropertyDescriptor.Builder()
             .name("schema-access-strategy")
             .displayName("Schema Access Strategy")
@@ -69,6 +61,15 @@ public class SchemaAccessUtils {
             .required(true)
             .build();
 
+    public static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder()
+            .name("schema-registry")
+            .displayName("Schema Registry")
+            .description("Specifies the Controller Service to use for the Schema Registry")
+            .identifiesControllerService(SchemaRegistry.class)
+            .required(false)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, SCHEMA_NAME_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA, CONFLUENT_ENCODED_SCHEMA)
+            .build();
+
     public static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder()
             .name("schema-name")
             .displayName("Schema Name")
@@ -76,6 +77,7 @@ public class SchemaAccessUtils {
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${schema.name}")
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, SCHEMA_NAME_PROPERTY)
             .required(false)
             .build();
 
@@ -86,6 +88,7 @@ public class SchemaAccessUtils {
                     "If the chosen Schema Registry does not support branching, this value will be ignored.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
             .required(false)
             .build();
 
@@ -96,6 +99,7 @@ public class SchemaAccessUtils {
                     "If not specified then the latest version of the schema will be retrieved.")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
             .required(false)
             .build();
 
@@ -106,6 +110,7 @@ public class SchemaAccessUtils {
             .addValidator(new AvroSchemaValidator())
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${avro.schema}")
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, SCHEMA_TEXT_PROPERTY)
             .required(false)
             .build();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDependencyDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDependencyDTO.java
new file mode 100644
index 0000000..96d8dc8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDependencyDTO.java
@@ -0,0 +1,47 @@
+/*
+ * 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.web.api.dto;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.Set;
+
+@XmlType(name = "propertyDependency")
+public class PropertyDependencyDTO {
+    private String propertyName;
+    private Set<String> dependentValues;
+
+    @ApiModelProperty("The name of the property that is being depended upon")
+    public String getPropertyName() {
+        return propertyName;
+    }
+
+    public void setPropertyName(final String propertyName) {
+        this.propertyName = propertyName;
+    }
+
+    @ApiModelProperty("The values for the property that satisfies the dependency, or null if the dependency is satisfied by the presence of any value for the associated property name")
+    public Set<String> getDependentValues() {
+        return dependentValues;
+    }
+
+    public void setDependentValues(final Set<String> dependentValues) {
+        this.dependentValues = dependentValues;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
index 5834f15..abc4677 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
@@ -40,6 +40,7 @@ public class PropertyDescriptorDTO {
     private String expressionLanguageScope;
     private String identifiesControllerService;
     private BundleDTO identifiesControllerServiceBundle;
+    private List<PropertyDependencyDTO> dependencies;
 
     /**
      * @return set of allowable values for this property. If empty then the allowable values are not constrained
@@ -208,4 +209,14 @@ public class PropertyDescriptorDTO {
     public void setIdentifiesControllerServiceBundle(BundleDTO identifiesControllerServiceBundle) {
         this.identifiesControllerServiceBundle = identifiesControllerServiceBundle;
     }
+
+    @ApiModelProperty(value="A list of dependencies that must be met in order for this Property to be relevant. If any of these dependencies is not met, the property described by this " +
+        "Property Descriptor is not relevant.")
+    public List<PropertyDependencyDTO> getDependencies() {
+        return dependencies;
+    }
+
+    public void setDependencies(final List<PropertyDependencyDTO> dependencies) {
+        this.dependencies = dependencies;
+    }
 }
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 ff858fc..5437d44 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
@@ -61,6 +61,7 @@ import org.apache.nifi.cluster.event.NodeEvent;
 import org.apache.nifi.cluster.manager.StatusMerger;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDependency;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
@@ -4065,6 +4066,20 @@ public final class DtoFactory {
             dto.setAllowableValues(allowableValues);
         }
 
+        // Add any dependencies
+        final Set<PropertyDependency> dependencies = propertyDescriptor.getDependencies();
+        final List<PropertyDependencyDTO> dependencyDtos = dependencies.stream()
+            .map(this::createPropertyDependencyDto)
+            .collect(Collectors.toList());
+        dto.setDependencies(dependencyDtos);
+
+        return dto;
+    }
+
+    private PropertyDependencyDTO createPropertyDependencyDto(final PropertyDependency dependency) {
+        final PropertyDependencyDTO dto = new PropertyDependencyDTO();
+        dto.setPropertyName(dependency.getPropertyName());
+        dto.setDependentValues(dependency.getDependentValues());
         return dto;
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
index 2fdb4ba..60fb4c3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
@@ -16,26 +16,9 @@
  */
 package org.apache.nifi.processors.standard;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.zip.Deflater;
-import java.util.zip.DeflaterOutputStream;
-import java.util.zip.InflaterInputStream;
-
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
 import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
@@ -54,7 +37,6 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
@@ -76,9 +58,25 @@ import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
 import org.xerial.snappy.SnappyInputStream;
 import org.xerial.snappy.SnappyOutputStream;
 
-import lzma.sdk.lzma.Decoder;
-import lzma.streams.LzmaInputStream;
-import lzma.streams.LzmaOutputStream;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
 
 @EventDriven
 @SideEffectFree
@@ -111,47 +109,50 @@ public class CompressContent extends AbstractProcessor {
     public static final String MODE_DECOMPRESS = "decompress";
 
     public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
-    .name("Compression Format")
-    .description("The compression format to use. Valid values are: GZIP, Deflate, BZIP2, XZ-LZMA2, LZMA, Snappy, Snappy Hadoop, Snappy Framed, and LZ4-Framed")
-    .allowableValues(COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+        .name("Compression Format")
+        .description("The compression format to use. Valid values are: GZIP, Deflate, BZIP2, XZ-LZMA2, LZMA, Snappy, Snappy Hadoop, Snappy Framed, and LZ4-Framed")
+        .allowableValues(COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
             COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
             COMPRESSION_FORMAT_LZ4_FRAMED)
-    .defaultValue(COMPRESSION_FORMAT_ATTRIBUTE)
-    .required(true)
-    .build();
+        .defaultValue(COMPRESSION_FORMAT_ATTRIBUTE)
+        .required(true)
+        .build();
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+        .name("Mode")
+        .description("Indicates whether the processor should compress content or decompress content. Must be either 'compress' or 'decompress'")
+        .allowableValues(MODE_COMPRESS, MODE_DECOMPRESS)
+        .defaultValue(MODE_COMPRESS)
+        .required(true)
+        .build();
     public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
-    .name("Compression Level")
-    .description("The compression level to use; this is valid only when using gzip, deflate or xz-lzma2 compression. A lower value results in faster processing "
-        + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
-        + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+        .name("Compression Level")
+        .description("The compression level to use; this is valid only when using gzip, deflate or xz-lzma2 compression. A lower value results in faster processing "
+            + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+            + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
         .defaultValue("1")
         .required(true)
         .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+        .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_XZ_LZMA2)
+        .dependsOn(MODE, MODE_COMPRESS)
         .build();
-    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
-    .name("Mode")
-    .description("Indicates whether the processor should compress content or decompress content. Must be either 'compress' or 'decompress'.")
-    .allowableValues(MODE_COMPRESS, MODE_DECOMPRESS)
-    .defaultValue(MODE_COMPRESS)
-    .required(true)
-    .build();
+
     public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
-    .name("Update Filename")
-    .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
-        + "compression format) and add the appropriate extension when compressing data")
+        .name("Update Filename")
+        .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+            + "compression format) and add the appropriate extension when compressing data")
         .required(true)
         .allowableValues("true", "false")
         .defaultValue("false")
         .build();
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
-    .name("success")
-    .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
-    .build();
+        .name("success")
+        .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+        .build();
     public static final Relationship REL_FAILURE = new Relationship.Builder()
-    .name("failure")
-    .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
-    .build();
+        .name("failure")
+        .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+        .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -200,7 +201,6 @@ public class CompressContent extends AbstractProcessor {
     protected Collection<ValidationResult> customValidate(final ValidationContext context) {
         final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(context));
 
-        final Validator rateValidator;
         if (context.getProperty(COMPRESSION_FORMAT).getValue().toLowerCase().equals(COMPRESSION_FORMAT_SNAPPY_HADOOP)
                 && context.getProperty(MODE).getValue().toLowerCase().equals(MODE_DECOMPRESS)) {
             validationResults.add(new ValidationResult.Builder().subject(COMPRESSION_FORMAT.getName())


[nifi] 04/07: NIFI-1121: Added an additional check for hidden properties to account for transitive dependent properties. - Added a 'dependent' attribute to determine whether or not to save dependent property values

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 535cab3167a5c2cd88ef560494628186317bf0d3
Author: mtien <mt...@gmail.com>
AuthorDate: Thu Aug 27 16:35:30 2020 -0700

    NIFI-1121: Added an additional check for hidden properties to account for transitive dependent properties.
    - Added a 'dependent' attribute to determine whether or not to save dependent property values
    
    Co-authored-by: Scott Aslan <sc...@gmail.com>
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../apache/nifi/components/PropertyDependency.java |  4 +-
 .../apache/nifi/processor/util/bin/BinFiles.java   |  3 +-
 .../jquery/propertytable/jquery.propertytable.js   | 54 ++++++++++++++++------
 .../nifi/processors/standard/MergeContent.java     |  5 +-
 4 files changed, 46 insertions(+), 20 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
index 5c0f9b5..8e09fc0 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
@@ -75,8 +75,8 @@ public class PropertyDependency {
         }
 
         final PropertyDependency that = (PropertyDependency) o;
-        return Objects.equals(getPropertyName(), that.getPropertyName()) &&
-            Objects.equals(getDependentValues(), that.getDependentValues());
+        return Objects.equals(getPropertyName(), that.getPropertyName())
+                && Objects.equals(getDependentValues(), that.getDependentValues());
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
index 4ba4a96..7851654 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
@@ -26,7 +26,6 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -51,7 +50,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
 
     public static final PropertyDescriptor MIN_SIZE = new PropertyDescriptor.Builder()
             .name("Minimum Group Size")
-            .description("The minimum size of for the bundle")
+            .description("The minimum size for the bundle")
             .required(true)
             .defaultValue("0 B")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
index 7767508..e5e1208 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
@@ -1674,6 +1674,7 @@
                         // Get the property descriptor object
                         var descriptor = descriptors[item.property];
                         var hidden = false;
+                        var dependent = false;
 
                         // Check for dependencies
                         if (descriptor.dependencies.length > 0) {
@@ -1687,12 +1688,17 @@
                                 // Check the row's dependent values against all other row's current values to determine hidden state
                                 $.each(items, function (k, property) {
                                     if (property.property === dependency.propertyName) {
-                                        // Get the current property value to compare with the dependent value
-                                        var propertyValue = property.value;
-
-                                        // Test the dependentValues array against the current value of the property
-                                        // If not, then mark the current property hidden attribute is true
-                                        hidden = !dependency.dependentValues.includes(propertyValue);
+                                        dependent = true;
+                                        if (property.hidden === false) {
+                                            // Get the current property value to compare with the dependent value
+                                            var propertyValue = property.value;
+
+                                            // Test the dependentValues array against the current value of the property
+                                            // If not, then mark the current property hidden attribute is true
+                                            hidden = !dependency.dependentValues.includes(propertyValue);
+                                        } else {
+                                            hidden = true;
+                                        }
                                         if (hidden) {
                                             // It is sufficient to have found a single instance of not meeting the
                                             // requirement for a dependent value in order to hide a property
@@ -1705,7 +1711,8 @@
 
                         propertyData.beginUpdate();
                         propertyData.updateItem(id, $.extend(item, {
-                            hidden: hidden
+                            hidden: hidden,
+                            dependent: dependent
                         }));
                         propertyData.endUpdate();
 
@@ -1826,18 +1833,38 @@
                 }
 
                 var hidden = false;
+                var dependent = false;
 
                 // Check for dependencies
                 if (descriptor.dependencies.length > 0) {
                     $.each(descriptor.dependencies, function (i, dependency) {
-                        // Get the property value by propertyName
-                        var propertyValue = properties[dependency.propertyName];
-                        // Test the dependentValues against the current value of the property
-                        // If not, then mark the current property hidden attribute is true
-                        hidden = !dependency.dependentValues.includes(propertyValue);
+                        // It is sufficient to have found a single instance of not meeting the
+                        // requirement for a dependent value in order to hide a property
                         if (hidden) {
                             return false;
                         }
+
+                        // Get the rows from the table
+                        var items = propertyData.getItems();
+
+                        // Get the item's hidden attribute to compare. If item.hidden=true, hidden = true.
+                        $.each(items, function (k, property) {
+                            if (property.property === dependency.propertyName) {
+                                dependent = true;
+                                if (property.hidden === false) {
+                                    // Get the property value by propertyName
+                                    var propertyValue = properties[dependency.propertyName];
+                                    // Test the dependentValues against the current value of the property
+                                    // If not, then mark the current property hidden attribute is true
+                                    hidden = !dependency.dependentValues.includes(propertyValue);
+                                } else {
+                                    hidden = true;
+                                }
+                                if (hidden) {
+                                    return false;
+                                }
+                            }
+                        })
                     });
                 }
 
@@ -1845,6 +1872,7 @@
                 propertyData.addItem({
                     id: i++,
                     hidden: hidden,
+                    dependent: dependent,
                     property: name,
                     displayName: displayName,
                     previousValue: value,
@@ -2221,7 +2249,7 @@
                 var propertyGrid = table.data('gridInstance');
                 var propertyData = propertyGrid.getData();
                 $.each(propertyData.getItems(), function () {
-                    if (this.hidden === true) {
+                    if (this.hidden === true && !(this.dependent === true)) {
                         // hidden properties were removed by the user, clear the value
                         properties[this.property] = null;
                     } else if (this.value !== this.previousValue) {
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 215822e..13eb119 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -43,7 +43,6 @@ import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipException;
 import java.util.zip.ZipOutputStream;
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.CodecFactory;
 import org.apache.avro.file.DataFileConstants;
@@ -55,13 +54,13 @@ import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
 import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
 import org.apache.nifi.annotation.behavior.ReadsAttributes;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
@@ -284,7 +283,7 @@ public class MergeContent extends BinFiles {
                     + "the values of the properties should be used as the content.")
             .allowableValues(DELIMITER_STRATEGY_NONE, DELIMITER_STRATEGY_FILENAME, DELIMITER_STRATEGY_TEXT)
             .defaultValue(DELIMITER_STRATEGY_NONE.getValue())
-            .dependsOn(MERGE_STRATEGY, MERGE_STRATEGY_BIN_PACK)
+            .dependsOn(MERGE_FORMAT, MERGE_FORMAT_CONCAT_VALUE)
             .build();
     public static final PropertyDescriptor HEADER = new PropertyDescriptor.Builder()
             .name("Header File")


[nifi] 06/07: NIFI-1121 Fixed a dependent value check error.

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 42c2cda9a21fc1d66040c6b95dcff1164515e2a6
Author: mtien <mt...@gmail.com>
AuthorDate: Thu Oct 29 09:02:15 2020 -0700

    NIFI-1121 Fixed a dependent value check error.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../js/jquery/propertytable/jquery.propertytable.js     | 17 +++++++++++++++--
 .../processors/tests/system/DependOnProperties.java     |  7 +++----
 2 files changed, 18 insertions(+), 6 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
index e5e1208..4a3b37d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
@@ -1695,7 +1695,13 @@
 
                                             // Test the dependentValues array against the current value of the property
                                             // If not, then mark the current property hidden attribute is true
-                                            hidden = !dependency.dependentValues.includes(propertyValue);
+                                            if (propertyValue != null) {
+                                                if (dependency.hasOwnProperty("dependentValues")) {
+                                                    hidden = !dependency.dependentValues.includes(propertyValue);
+                                                }
+                                            } else {
+                                                hidden = true;
+                                            }
                                         } else {
                                             hidden = true;
                                         }
@@ -1854,9 +1860,16 @@
                                 if (property.hidden === false) {
                                     // Get the property value by propertyName
                                     var propertyValue = properties[dependency.propertyName];
+
                                     // Test the dependentValues against the current value of the property
                                     // If not, then mark the current property hidden attribute is true
-                                    hidden = !dependency.dependentValues.includes(propertyValue);
+                                    if (propertyValue != null) {
+                                        if (dependency.hasOwnProperty("dependentValues")) {
+                                            hidden = !dependency.dependentValues.includes(propertyValue);
+                                        }
+                                    } else {
+                                        hidden = true;
+                                    }
                                 } else {
                                     hidden = true;
                                 }
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
index 49c2dbb..50135e7 100644
--- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
@@ -17,6 +17,8 @@
 
 package org.apache.nifi.processors.tests.system;
 
+import java.util.Arrays;
+import java.util.List;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyDescriptor.Builder;
@@ -26,9 +28,6 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
-import java.util.Arrays;
-import java.util.List;
-
 import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
 
 public class DependOnProperties extends AbstractProcessor {
@@ -74,7 +73,7 @@ public class DependOnProperties extends AbstractProcessor {
     static final PropertyDescriptor REQUIRED_IF_ALWAYS_REQUIRED_IS_BAR_OR_BAZ = new Builder()
         .name("Required If Always Required Is Bar Or Baz")
         .displayName("Required If Always Required Is Bar Or Baz")
-        .description("This property is required if and only if hte 'Always Required' property is set to the value 'bar' or the value 'baz'")
+        .description("This property is required if and only if the 'Always Required' property is set to the value 'bar' or the value 'baz'")
         .required(true)
         .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
         .dependsOn(ALWAYS_REQUIRED, BAR, BAZ)


[nifi] 02/07: NIFI-1121 Show and hide properties that depend on another property.

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 4bd9d7b4139bb05db96a15c9cf770ccd3c33bb38
Author: mtien <mt...@gmail.com>
AuthorDate: Fri Aug 14 11:14:10 2020 -0700

    NIFI-1121 Show and hide properties that depend on another property.
    
    Co-authored-by: Scott Aslan <sc...@gmail.com>
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../jquery/propertytable/jquery.propertytable.js   | 75 +++++++++++++++++++++-
 1 file changed, 73 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
index 6380524..7767508 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js
@@ -1661,6 +1661,58 @@
         if (options.readOnly !== true) {
             propertyGrid.onBeforeCellEditorDestroy.subscribe(function (e, args) {
                 setTimeout(function() {
+                    var propertyData = propertyGrid.getData();
+
+                    // Get the default properties object
+                    var descriptors = table.data('descriptors');
+
+                    // Get the rows from the table
+                    var items = propertyData.getItems();
+
+                    // Loop over each row
+                    $.each(items, function (id, item) {
+                        // Get the property descriptor object
+                        var descriptor = descriptors[item.property];
+                        var hidden = false;
+
+                        // Check for dependencies
+                        if (descriptor.dependencies.length > 0) {
+                            // Loop over each dependency
+                            $.each(descriptor.dependencies, function (i, dependency) {
+                                // It is sufficient to have found a single instance of not meeting the
+                                // requirement for a dependent value in order to hide a property
+                                if (hidden) {
+                                    return false;
+                                }
+                                // Check the row's dependent values against all other row's current values to determine hidden state
+                                $.each(items, function (k, property) {
+                                    if (property.property === dependency.propertyName) {
+                                        // Get the current property value to compare with the dependent value
+                                        var propertyValue = property.value;
+
+                                        // Test the dependentValues array against the current value of the property
+                                        // If not, then mark the current property hidden attribute is true
+                                        hidden = !dependency.dependentValues.includes(propertyValue);
+                                        if (hidden) {
+                                            // It is sufficient to have found a single instance of not meeting the
+                                            // requirement for a dependent value in order to hide a property
+                                            return false;
+                                        }
+                                    }
+                                })
+                            });
+                        }
+
+                        propertyData.beginUpdate();
+                        propertyData.updateItem(id, $.extend(item, {
+                            hidden: hidden
+                        }));
+                        propertyData.endUpdate();
+
+                        // Reset hidden property
+                        hidden = false;
+                    });
+
                     propertyGrid.resizeCanvas();
                 }, 50);
             });
@@ -1733,7 +1785,7 @@
      * @param {type} history
      */
     var loadProperties = function (table, properties, descriptors, history) {
-        // save the descriptors and history
+        // save the original descriptors and history
         table.data({
             'descriptors': descriptors,
             'history': history
@@ -1773,16 +1825,35 @@
                     }
                 }
 
+                var hidden = false;
+
+                // Check for dependencies
+                if (descriptor.dependencies.length > 0) {
+                    $.each(descriptor.dependencies, function (i, dependency) {
+                        // Get the property value by propertyName
+                        var propertyValue = properties[dependency.propertyName];
+                        // Test the dependentValues against the current value of the property
+                        // If not, then mark the current property hidden attribute is true
+                        hidden = !dependency.dependentValues.includes(propertyValue);
+                        if (hidden) {
+                            return false;
+                        }
+                    });
+                }
+
                 // add the row
                 propertyData.addItem({
                     id: i++,
-                    hidden: false,
+                    hidden: hidden,
                     property: name,
                     displayName: displayName,
                     previousValue: value,
                     value: value,
                     type: type
                 });
+
+                // Reset hidden property
+                hidden = false;
             });
 
             propertyData.endUpdate();


[nifi] 05/07: NIFI-1121: Updated backend to perform appropriate validation. Added tests. Updated documentation writer. Updated dev guide to explain how PropertyDescriptor.Builder#dependsOn affects validation. Updated JavaDocs for PropertyDescriptor.Builder#dependsOn

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 4b9014b9596a8f479a32d0b3e52bc6649b0f0d1b
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Sep 4 15:26:54 2020 -0400

    NIFI-1121: Updated backend to perform appropriate validation. Added tests. Updated documentation writer. Updated dev guide to explain how PropertyDescriptor.Builder#dependsOn affects validation. Updated JavaDocs for PropertyDescriptor.Builder#dependsOn
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../components/AbstractConfigurableComponent.java  |  74 ++++++++++--
 .../apache/nifi/components/PropertyDependency.java |  18 ++-
 .../apache/nifi/components/PropertyDescriptor.java |  58 +++++-----
 .../apache/nifi/components/ValidationContext.java  |  23 +++-
 .../documentation/xml/XmlDocumentationWriter.java  |  30 +++++
 .../NotificationValidationContext.java             |   6 +
 nifi-docs/src/main/asciidoc/developer-guide.adoc   |  37 +++++-
 nifi-mock/pom.xml                                  |   5 +
 .../org/apache/nifi/util/MockProcessContext.java   |  50 +++++----
 .../org/apache/nifi/util/MockPropertyValue.java    |  28 ++---
 .../apache/nifi/util/MockValidationContext.java    | 124 +++++++++++++++++++--
 .../nifi/util/StandardProcessorTestRunner.java     |  15 ++-
 .../main/java/org/apache/nifi/util/TestRunner.java |  29 ++---
 .../SchemaRegistryRecordSetWriter.java             |  28 ++---
 .../html/HtmlDocumentationWriter.java              |  47 ++++++++
 .../nifi-framework-components/pom.xml              |  41 +++++++
 .../org/apache/nifi/AbstractValidationContext.java | 110 ++++++++++++++++++
 .../nifi/controller/AbstractComponentNode.java     |   9 ++
 .../nifi-framework/nifi-framework-core/pom.xml     |   5 +
 .../nifi/processor/StandardValidationContext.java  |   6 +-
 .../nifi/controller/TestStandardProcessorNode.java |   7 ++
 .../nifi-framework/nifi-stateless/pom.xml          |   5 +
 .../stateless/core/StatelessProcessContext.java    |   4 +
 .../stateless/core/StatelessValidationContext.java |  20 +++-
 .../nifi-framework-bundle/nifi-framework/pom.xml   |   1 +
 .../nifi/script/impl/ValidationContextAdapter.java |   6 +
 .../nifi/processors/standard/MergeContent.java     |   3 +-
 .../nifi/processors/standard/TestMergeContent.java |  36 +++---
 .../processors/standard/TestValidateRecord.java    |   1 -
 .../org/apache/nifi/json/JsonRecordSetWriter.java  |   1 +
 .../java/org/apache/nifi/json/JsonTreeReader.java  |   9 +-
 .../jetty/ControllerServiceTestContext.java        |   4 +
 .../tests/system/DependOnProperties.java           | 118 ++++++++++++++++++++
 .../services/org.apache.nifi.processor.Processor   |   1 +
 .../validation/DependentPropertyValidationIT.java  |  99 ++++++++++++++++
 35 files changed, 911 insertions(+), 147 deletions(-)

diff --git a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
index 25ffae5..4758802 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public abstract class AbstractConfigurableComponent implements ConfigurableComponent {
 
@@ -57,15 +58,10 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
     }
 
     private PropertyDescriptor getPropertyDescriptor(final PropertyDescriptor specDescriptor) {
-        PropertyDescriptor descriptor = null;
         //check if property supported
-        final List<PropertyDescriptor> supportedDescriptors = getSupportedPropertyDescriptors();
-        if (supportedDescriptors != null) {
-            for (final PropertyDescriptor desc : supportedDescriptors) { //find actual descriptor
-                if (specDescriptor.equals(desc)) {
-                    return desc;
-                }
-            }
+        PropertyDescriptor descriptor = getSupportedPropertyDescriptor(specDescriptor);
+        if (descriptor != null) {
+            return descriptor;
         }
 
         descriptor = getSupportedDynamicPropertyDescriptor(specDescriptor.getName());
@@ -79,6 +75,19 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
         return descriptor;
     }
 
+    private PropertyDescriptor getSupportedPropertyDescriptor(final PropertyDescriptor specDescriptor) {
+        final List<PropertyDescriptor> supportedDescriptors = getSupportedPropertyDescriptors();
+        if (supportedDescriptors != null) {
+            for (final PropertyDescriptor desc : supportedDescriptors) { //find actual descriptor
+                if (specDescriptor.equals(desc)) {
+                    return desc;
+                }
+            }
+        }
+
+        return null;
+    }
+
     @Override
     public final Collection<ValidationResult> validate(final ValidationContext context) {
         // goes through supported properties
@@ -87,6 +96,15 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
 
         if (null != supportedDescriptors) {
             for (final PropertyDescriptor descriptor : supportedDescriptors) {
+                // If the property descriptor's dependency is not satisfied, the property does not need to be considered, as it's not relevant to the
+                // component's functionality.
+                final boolean dependencySatisfied = context.isDependencySatisfied(descriptor, this::getPropertyDescriptor);
+                if (!dependencySatisfied) {
+                    continue;
+                }
+
+                validateDependencies(descriptor, context, results);
+
                 String value = context.getProperty(descriptor).getValue();
                 if (value == null) {
                     value = descriptor.getDefaultValue();
@@ -138,6 +156,41 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
         return results;
     }
 
+    private void validateDependencies(final PropertyDescriptor descriptor, final ValidationContext context, final Collection<ValidationResult> results) {
+        // Ensure that we don't have any dependencies on non-existent properties.
+        final Set<PropertyDependency> dependencies = descriptor.getDependencies();
+        for (final PropertyDependency dependency : dependencies) {
+            final String dependentPropertyName = dependency.getPropertyName();
+
+            // If there's a supported property descriptor then all is okay.
+            final PropertyDescriptor specDescriptor = new PropertyDescriptor.Builder().name(dependentPropertyName).build();
+            final PropertyDescriptor supportedDescriptor = getSupportedPropertyDescriptor(specDescriptor);
+            if (supportedDescriptor != null) {
+                continue;
+            }
+
+            final PropertyDescriptor dynamicPropertyDescriptor = getSupportedDynamicPropertyDescriptor(dependentPropertyName);
+            if (dynamicPropertyDescriptor == null) {
+                results.add(new ValidationResult.Builder()
+                    .subject(descriptor.getDisplayName())
+                    .valid(false)
+                    .explanation("Property depends on property " + dependentPropertyName + ", which is not a known property")
+                    .build());
+            }
+
+            // Dependent property is supported as a dynamic property. This is okay as long as there is a value set.
+            final PropertyValue value = context.getProperty(dynamicPropertyDescriptor);
+            if (value == null || !value.isSet()) {
+                results.add(new ValidationResult.Builder()
+                    .subject(descriptor.getDisplayName())
+                    .valid(false)
+                    .explanation("Property depends on property " + dependentPropertyName + ", which is not a known property")
+                    .build());
+            }
+        }
+
+    }
+
     /**
      * Hook method allowing subclasses to eagerly react to a configuration
      * change for the given property descriptor. As an alternative to using this
@@ -185,15 +238,14 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo
      *
      * @return PropertyDescriptor objects this processor currently supports
      */
-    @SuppressWarnings("unchecked")
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return Collections.EMPTY_LIST;
+        return Collections.emptyList();
     }
 
     @Override
     public final List<PropertyDescriptor> getPropertyDescriptors() {
         final List<PropertyDescriptor> supported = getSupportedPropertyDescriptors();
-        return supported == null ? Collections.<PropertyDescriptor>emptyList() : new ArrayList<>(supported);
+        return supported == null ? Collections.emptyList() : new ArrayList<>(supported);
     }
 
     @Override
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
index 8e09fc0..275248e 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDependency.java
@@ -24,24 +24,29 @@ import java.util.Set;
 
 public class PropertyDependency {
     private final String propertyName;
+    private final String displayName;
     private final Set<String> dependentValues;
 
     /**
      * Creates a dependency that is satisfied if any value is set for the property with the given name
      * @param propertyName the name of the property that is depended upon
+     * @param propertyDisplayName the display name of the property that is depended upon
      */
-    public PropertyDependency(final String propertyName) {
+    public PropertyDependency(final String propertyName, final String propertyDisplayName) {
         this.propertyName = Objects.requireNonNull(propertyName);
+        this.displayName = propertyDisplayName == null ? propertyName : propertyDisplayName;
         this.dependentValues = null;
     }
 
     /**
      * Creates a dependency that is satisfied only if the property with the given name has a value that is in the given set of dependent values
      * @param propertyName the name of the property that is depended upon
+     * @param propertyDisplayName the display name of the property that is depended upon
      * @param dependentValues the values that satisfy the dependency
      */
-    public PropertyDependency(final String propertyName, final Set<String> dependentValues) {
+    public PropertyDependency(final String propertyName, final String propertyDisplayName, final Set<String> dependentValues) {
         this.propertyName = Objects.requireNonNull(propertyName);
+        this.displayName = propertyDisplayName == null ? propertyName : propertyDisplayName;
         this.dependentValues = Collections.unmodifiableSet(new HashSet<>(Objects.requireNonNull(dependentValues)));
     }
 
@@ -53,6 +58,13 @@ public class PropertyDependency {
     }
 
     /**
+     * @return the display name of the property that is depended upon
+     */
+    public String getPropertyDisplayName() {
+        return displayName;
+    }
+
+    /**
      * @return the Set of values that satisfy the dependency
      */
     public Set<String> getDependentValues() {
@@ -61,7 +73,7 @@ public class PropertyDependency {
 
     @Override
     public String toString() {
-        return "PropertyDependency[propertyName=" + propertyName + ", dependentValues=" + dependentValues + "]";
+        return "PropertyDependency[propertyName=" + propertyName + ", displayName=" + displayName + ", dependentValues=" + dependentValues + "]";
     }
 
     @Override
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
index 012a702..29b6b67 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java
@@ -458,6 +458,13 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
          * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
          * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
          *
+         * In the case that this property is NOT considered to be relevant (meaning that it depends on a property whose value is not specified, or whose value does not match one of the given
+         * Allowable Values), the property will not be shown in the component's configuration in the User Interface. Additionally, this property's value will not be considered for
+         * validation. That is, if this property is configured with an invalid value and this property depends on Property Foo, and Property Foo does not have a value set, then the component
+         * will still be valid, because the value of this property is irrelevant.
+         *
+         * If the given property is not relevant (because its dependencies are not satisfied), this property is also considered not to be valid.
+         *
          * @param property the property that must be set in order for this property to become relevant
          * @param dependentValues the possible values for the given property for which this Property is relevant
          * @return the builder
@@ -468,14 +475,14 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
             }
 
             if (dependentValues.length == 0) {
-                dependencies.add(new PropertyDependency(property.getName()));
+                dependencies.add(new PropertyDependency(property.getName(), property.getDisplayName()));
             } else {
                 final Set<String> dependentValueSet = new HashSet<>();
                 for (final AllowableValue value : dependentValues) {
                     dependentValueSet.add(value.getValue());
                 }
 
-                dependencies.add(new PropertyDependency(property.getName(), dependentValueSet));
+                dependencies.add(new PropertyDependency(property.getName(), property.getDisplayName(), dependentValueSet));
             }
 
             return this;
@@ -483,44 +490,33 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
 
 
         /**
-         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a non-null value.
-         * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
+         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a value equal to one of the given
+         * <code>String</code> arguments.
          * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
          *
-         * @param property the property that must be set in order for this property to become relevant
-         * @param dependentValues the possible values for the given property for which this Property is relevant
-         * @return the builder
-         */
-        public Builder dependsOn(final PropertyDescriptor property, final String... dependentValues) {
-            return dependsOn(property.getName(), dependentValues);
-        }
-
-
-        /**
-         * Establishes a relationship between this Property and the given property by declaring that this Property is only relevant if the given Property has a non-null value.
-         * Furthermore, if one or more explicit Allowable Values are provided, this Property will not be relevant unless the given Property's value is equal to one of the given Allowable Values.
-         * If this method is called multiple times, each with a different dependency, then a relationship is established such that this Property is relevant only if all dependencies are satisfied.
+         * In the case that this property is NOT considered to be relevant (meaning that it depends on a property whose value is not specified, or whose value does not match one of the given
+         * Allowable Values), the property will not be shown in the component's configuration in the User Interface. Additionally, this property's value will not be considered for
+         * validation. That is, if this property is configured with an invalid value and this property depends on Property Foo, and Property Foo does not have a value set, then the component
+         * will still be valid, because the value of this property is irrelevant.
          *
-         * @param propertyName the property that must be set in order for this property to become relevant
-         * @param dependentValues the possible values for the given property for which this Property is relevant
+         * If the given property is not relevant (because its dependencies are not satisfied), this property is also considered not to be valid.
+         *
+         * @param property the property that must be set in order for this property to become relevant
+         * @param firstDependentValue the first value for the given property for which this Property is relevant
+         * @param additionalDependentValues any other values for the given property for which this Property is relevant
          * @return the builder
          */
-        public Builder dependsOn(final String propertyName, final String... dependentValues) {
-            if (dependencies == null) {
-                dependencies = new HashSet<>();
+        public Builder dependsOn(final PropertyDescriptor property, final String firstDependentValue, final String... additionalDependentValues) {
+            final AllowableValue[] dependentValues = new AllowableValue[additionalDependentValues.length + 1];
+            dependentValues[0] = new AllowableValue(firstDependentValue);
+            int i=1;
+            for (final String additionalDependentValue : additionalDependentValues) {
+                dependentValues[i++] = new AllowableValue(additionalDependentValue);
             }
 
-            if (dependentValues.length == 0) {
-                dependencies.add(new PropertyDependency(propertyName));
-            } else {
-                final Set<String> dependentValueSet = new HashSet<>(Arrays.asList(dependentValues));
-                dependencies.add(new PropertyDependency(propertyName, dependentValueSet));
-            }
-
-            return this;
+            return dependsOn(property, dependentValues);
         }
 
-
         /**
          * @return a PropertyDescriptor as configured
          *
diff --git a/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java b/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
index 56f566e..81543d7 100644
--- a/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java
@@ -16,13 +16,15 @@
  */
 package org.apache.nifi.components;
 
-import java.util.Collection;
-import java.util.Map;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
 
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Function;
+
 public interface ValidationContext extends PropertyContext {
 
     /**
@@ -113,4 +115,21 @@ public interface ValidationContext extends PropertyContext {
      * @return <code>true</code> if the Parameter is defined and has a non-null value, false otherwise
      */
     boolean isParameterSet(String parameterName);
+
+    /**
+     * Determines whether or not the dependencies of the given Property Descriptor are satisfied.
+     * If the given Property Descriptor has no dependency on another property, then the dependency is satisfied.
+     * If there is at least one dependency, then all dependencies must be satisfied.
+     * In order for a dependency to be considered satisfied, all of the following must be true:
+     * <ul>
+     *     <li>The property that is depended upon has all of its dependencies satisfied.</li>
+     *     <li>If the given Property Descriptor depends on a given AllowableValue, then the property that is depended upon has a value that falls within the given range of Allowable Values for
+     *     the dependency.</li>
+     * </ul>
+     *
+     * @param propertyDescriptor the property descriptor
+     * @param propertyDescriptorLookup a lookup for converting from a property name to the property descriptor with that name
+     * @return <code>true</code> if all dependencies of the given property descriptor are satisfied, <code>false</code> otherwise
+     */
+    boolean isDependencySatisfied(PropertyDescriptor propertyDescriptor, Function<String, PropertyDescriptor> propertyDescriptorLookup);
 }
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 01c0bdc..abaab37 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
@@ -42,6 +42,7 @@ import org.apache.nifi.annotation.documentation.DeprecationNotice;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDependency;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.RequiredPermission;
 import org.apache.nifi.documentation.AbstractDocumentationWriter;
@@ -176,6 +177,7 @@ public class XmlDocumentationWriter extends AbstractDocumentationWriter {
         writeTextElement("expressionLanguageScope", property.getExpressionLanguageScope() == null ? null : property.getExpressionLanguageScope().name());
         writeBooleanElement("dynamicallyModifiesClasspath", property.isDynamicClasspathModifier());
         writeBooleanElement("dynamic", property.isDynamic());
+        writeDependencies(property);
 
         writeEndElement();
     }
@@ -188,6 +190,34 @@ public class XmlDocumentationWriter extends AbstractDocumentationWriter {
         writeEndElement();
     }
 
+    private void writeDependencies(final PropertyDescriptor propertyDescriptor) throws IOException {
+        final Set<PropertyDependency> dependencies = propertyDescriptor.getDependencies();
+        if (dependencies == null || dependencies.isEmpty()) {
+            return;
+        }
+
+        writeStartElement("dependencies");
+
+        for (final PropertyDependency dependency : dependencies) {
+            writeStartElement("dependency");
+            writeTextElement("propertyName", dependency.getPropertyName());
+            writeTextElement("propertyDisplayName", dependency.getPropertyDisplayName());
+
+            final Set<String> dependentValues = dependency.getDependentValues();
+            if (dependentValues != null) {
+                writeStartElement("dependentValues");
+                for (final String dependentValue : dependentValues) {
+                    writeTextElement("dependentValue", dependentValue);
+                }
+                writeEndElement();
+            }
+
+            writeEndElement();
+        }
+
+        writeEndElement();
+    }
+
     @Override
     protected void writeDynamicProperties(final List<DynamicProperty> dynamicProperties) throws IOException {
         writeArray("dynamicProperties", dynamicProperties, this::writeDynamicProperty);
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
index 0075df2..1cdb29b 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationValidationContext.java
@@ -35,6 +35,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
 
 public class NotificationValidationContext implements ValidationContext {
     private final NotificationContext context;
@@ -138,4 +139,9 @@ public class NotificationValidationContext implements ValidationContext {
     public boolean isParameterSet(final String parameterName) {
         return false;
     }
+
+    @Override
+    public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup) {
+        return true;
+    }
 }
diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc
index b16ce12..6152092 100644
--- a/nifi-docs/src/main/asciidoc/developer-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc
@@ -148,7 +148,7 @@ attribute within the `CoreAttributes` enum and how it appears in the UI/API.
 
 ===== Additional Common Attributes
 
-While these attributes are not members of the `CoreAttributes` enum, they are de facto standards across the system and found on most FlowFiles. 
+While these attributes are not members of the `CoreAttributes` enum, they are de facto standards across the system and found on most FlowFiles.
 
 - File Size (`fileSize`): The size of the FlowFile content in bytes.
 
@@ -351,6 +351,41 @@ restricting the allowable values for a
 property via the PropertyDescriptor.Builder's `allowableValues` method
 or `identifiesControllerService` method.
 
+In addition, if a property is dependent on another property (by means of the `PropertyDescriptor.Builder`'s
+`dependsOn` method) and the dependency is not satisfied, then the Property will be validated.
+
+For example, consider the following two Property Descriptors:
+```
+PropertyDescriptor USE_FILE = new PropertyDescriptor.Buildler()
+    .name("Use File")
+    .displayName("Use File")
+    .required(true)
+    .allowableValues("true", "false")
+    .defaultValue("true")
+    .build();
+```
+```
+PropertyDescriptor FILE = new PropertyDescriptor.Builder()
+    .name("File to Use")
+    .displayName("File to Use")
+    .required(true)
+    .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+    .dependsOn(USE_FILE, "true")
+    .build();
+```
+In this case, if the "Use File" property is set to `true`, then the Processor will not be valid unless the "File to Use" property is
+set to a valid filename. If "Use File" is set to `true` and "File to Use" does not have a value set, the Processor will be invalid
+(because the "File to Use" property is required). If "Use File" is set to `true` and "File to Use" has a value set but the specified
+file does not exist, the Processor will also be invalid because the "File to Use" property is invalid according to the Validator.
+
+However, if the "Use File" property is set to `false`, then the "File to Use" property is said to have a dependency that is not satisfied.
+As a result, the "File to Use" property will not be considered in the validation. Therefore, if "Use File" is set to `false` and "File to Use"
+has no value said, the Processor will still be valid (even though "File to Use" is required, it is only required if "Use File" is `true`).
+Likewise, if "File to Use" is set to a non-existent filename such as `/file/that/does/not/exist`, the Processor will still be valid so long as
+the "Use File" property is set to `false`.
+
+Furthermore, the "File to Use" property will not even be shown in the NiFi UI unless "Use File" is set to `true`.
+
 There are times, though, when validating a Processor's properties
 individually is not sufficient. For this purpose,
 the AbstractProcessor exposes a `customValidate` method. The method
diff --git a/nifi-mock/pom.xml b/nifi-mock/pom.xml
index 2047fa1..abafdb1 100644
--- a/nifi-mock/pom.xml
+++ b/nifi-mock/pom.xml
@@ -44,6 +44,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-components</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
             <version>1.13.0-SNAPSHOT</version>
             <exclusions>
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index ffc2711..6457e3b 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@ -90,25 +90,6 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
         this(component,null,stateManager,variableRegistry);
     }
 
-    /**
-     * Creates a new MockProcessContext for the given Processor with given name
-     *
-     * @param component being mocked
-     * @param componentName the name to be given the component;
-     * @param stateManager state manager
-     * @param variableRegistry variableRegistry
-     */
-    public MockProcessContext(final ConfigurableComponent component,
-                              final String componentName,
-                              final StateManager stateManager,
-                              final VariableRegistry variableRegistry) {
-        this.component = Objects.requireNonNull(component);
-        this.componentName = componentName == null ? "" : componentName;
-        this.inputRequirement = component.getClass().getAnnotation(InputRequirement.class);
-        this.stateManager = stateManager;
-        this.variableRegistry = variableRegistry;
-    }
-
     public MockProcessContext(final ControllerService component,
                               final MockProcessContext context,
                               final StateManager stateManager,
@@ -125,6 +106,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
 
         try {
             annotationData = context.getControllerServiceAnnotationData(component);
+
             final Map<PropertyDescriptor, String> props = context.getControllerServiceProperties(component);
             properties.putAll(props);
 
@@ -134,6 +116,27 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
         }
     }
 
+    /**
+     * Creates a new MockProcessContext for the given Processor with given name
+     *
+     * @param component being mocked
+     * @param componentName the name to be given the component;
+     * @param stateManager state manager
+     * @param variableRegistry variableRegistry
+     */
+    public MockProcessContext(final ConfigurableComponent component,
+                              final String componentName,
+                              final StateManager stateManager,
+                              final VariableRegistry variableRegistry) {
+        this.component = Objects.requireNonNull(component);
+        this.componentName = componentName == null ? "" : componentName;
+        this.inputRequirement = component.getClass().getAnnotation(InputRequirement.class);
+        this.stateManager = stateManager;
+        this.variableRegistry = variableRegistry;
+    }
+
+
+
     @Override
     public PropertyValue getProperty(final PropertyDescriptor descriptor) {
         return getProperty(descriptor.getName());
@@ -149,7 +152,8 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
         final String setPropertyValue = properties.get(descriptor);
         final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
 
-        return new MockPropertyValue(propValue, this, variableRegistry, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
+        final MockPropertyValue propertyValue = new MockPropertyValue(propValue, this, variableRegistry, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
+        return propertyValue;
     }
 
     @Override
@@ -161,6 +165,10 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
         return setProperty(new PropertyDescriptor.Builder().name(propertyName).build(), propertyValue);
     }
 
+    public PropertyDescriptor getPropertyDescriptor(final String propertyName) {
+        return component.getPropertyDescriptor(propertyName);
+    }
+
     /**
      * Updates the value of the property with the given PropertyDescriptor to
      * the specified value IF and ONLY IF the value is valid according to the
@@ -182,6 +190,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
         if (oldValue == null) {
             oldValue = fullyPopulatedDescriptor.getDefaultValue();
         }
+
         if ((value == null && oldValue != null) || (value != null && !value.equals(oldValue))) {
             component.onPropertyModified(fullyPopulatedDescriptor, oldValue, value);
         }
@@ -535,5 +544,4 @@ public class MockProcessContext extends MockControllerServiceLookup implements P
     public InputRequirement getInputRequirement() {
         return inputRequirement;
     }
-
 }
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
index 9b7d72b..296b75b 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java
@@ -75,7 +75,6 @@ public class MockPropertyValue implements PropertyValue {
         this.variableRegistry = variableRegistry;
     }
 
-
     private void ensureExpressionsEvaluated() {
         if (Boolean.TRUE.equals(expectExpressions) && !expressionsEvaluated) {
             throw new IllegalStateException("Attempting to retrieve value of " + propertyDescriptor
@@ -86,47 +85,38 @@ public class MockPropertyValue implements PropertyValue {
     }
 
     private void validateExpressionScope(boolean attributesAvailable) {
+        if (expressionLanguageScope == null) {
+            return;
+        }
+
         // language scope is not null, we have attributes available but scope is not equal to FF attributes
         // it means that we're not evaluating against flow file attributes even though attributes are available
-        if(expressionLanguageScope != null
-                && (attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
+        if(attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)) {
             throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
                     + " using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
                     + " proper scope should be set in the property descriptor using"
                     + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)");
         }
 
-        // if the service lookup is an instance of the validation context, we're in the validate() method
-        // at this point we don't have any flow file available and we should not care about the scope
-        // even though it is defined as FLOWFILE_ATTRIBUTES
-        if(expressionLanguageScope != null
-                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
-                && this.serviceLookup instanceof MockValidationContext) {
-            return;
-        }
 
         // we check if the input requirement is INPUT_FORBIDDEN
         // in that case, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
         // it likely means that the property has been defined in a common/abstract class used by multiple processors with
         // different input requirements.
-        if(expressionLanguageScope != null
-                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
-                && (this.serviceLookup.getInputRequirement() == null
-                    || this.serviceLookup.getInputRequirement().value().equals(InputRequirement.Requirement.INPUT_FORBIDDEN))) {
+        if(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
+                && (this.serviceLookup.getInputRequirement() == null || this.serviceLookup.getInputRequirement().value().equals(InputRequirement.Requirement.INPUT_FORBIDDEN))) {
             return;
         }
 
         // if we have a processor where input requirement is INPUT_ALLOWED, we need to check if there is an
         // incoming connection or not. If not, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
-        if(expressionLanguageScope != null
-                && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
+        if(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
                 && !((MockProcessContext) this.serviceLookup).hasIncomingConnection()) {
             return;
         }
 
         // we're trying to evaluate against flow files attributes but we don't have any attributes available.
-        if(expressionLanguageScope != null
-                && (!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
+        if(!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)) {
             throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
                     + " without using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
                     + " proper scope should be set in the property descriptor using"
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
index d4a198b..834833b 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java
@@ -16,13 +16,7 @@
  */
 package org.apache.nifi.util;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
+import org.apache.nifi.AbstractValidationContext;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
 import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler;
@@ -32,20 +26,34 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.PropertyConfiguration;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
 import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
 import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.ParameterParser;
 import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ParameterTokenList;
+import org.apache.nifi.parameter.StandardParameterTokenList;
 import org.apache.nifi.registry.VariableRegistry;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
 public class MockValidationContext extends MockControllerServiceLookup implements ValidationContext, ControllerServiceLookup {
 
     private final MockProcessContext context;
     private final Map<String, Boolean> expressionLanguageSupported;
     private final StateManager stateManager;
     private final VariableRegistry variableRegistry;
+    private final Map<PropertyDescriptor, PropertyConfiguration> properties;
 
     public MockValidationContext(final MockProcessContext processContext) {
         this(processContext, null, VariableRegistry.EMPTY_REGISTRY);
@@ -61,8 +69,25 @@ public class MockValidationContext extends MockControllerServiceLookup implement
         for (final PropertyDescriptor descriptor : properties.keySet()) {
             expressionLanguageSupported.put(descriptor.getName(), descriptor.isExpressionLanguageSupported());
         }
+
+        this.properties = createPropertyConfigurationMap(processContext);
     }
 
+    private static Map<PropertyDescriptor, PropertyConfiguration> createPropertyConfigurationMap(final MockProcessContext processContext) {
+        final Map<PropertyDescriptor, PropertyConfiguration> configurationMap = new HashMap<>();
+
+        for (final Map.Entry<String, String> entry : processContext.getAllProperties().entrySet()) {
+            final PropertyDescriptor descriptor = processContext.getPropertyDescriptor(entry.getKey());
+            final ParameterTokenList tokenList = new StandardParameterTokenList(entry.getValue(), Collections.emptyList());
+            final List<ParameterReference> parameterReferences = Collections.emptyList();
+            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), tokenList, parameterReferences);
+            configurationMap.put(descriptor, configuration);
+        }
+
+        return configurationMap;
+    }
+
+
     @Override
     public ControllerService getControllerService(final String identifier) {
         return context.getControllerService(identifier);
@@ -188,4 +213,89 @@ public class MockValidationContext extends MockControllerServiceLookup implement
         return true;
     }
 
+    @Override
+    public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup) {
+        // The logic for isDependencySatisfied is already implemented in the AbstractValidationContext. We don't want to duplicate the logic here.
+        // However, this class cannot extends AbstractValidationContext because it was already created to extend MockControllerServiceLookup. Changing that
+        // hierarchy gets complicated. So, for the time being we will create a concrete implementation of the AbstractValidationContext in order to call the
+        // isDependencySatisfied method.
+        final ValidationContext abstractValidationContext = new AbstractValidationContext(ParameterLookup.EMPTY, properties) {
+            @Override
+            public ControllerServiceLookup getControllerServiceLookup() {
+                return null;
+            }
+
+            @Override
+            public ValidationContext getControllerServiceValidationContext(final ControllerService controllerService) {
+                return null;
+            }
+
+            @Override
+            public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
+                return null;
+            }
+
+            @Override
+            public PropertyValue newPropertyValue(final String value) {
+                return null;
+            }
+
+            @Override
+            public Map<PropertyDescriptor, String> getProperties() {
+                return null;
+            }
+
+            @Override
+            public String getAnnotationData() {
+                return null;
+            }
+
+            @Override
+            public boolean isValidationRequired(final ControllerService service) {
+                return false;
+            }
+
+            @Override
+            public boolean isExpressionLanguagePresent(final String value) {
+                return false;
+            }
+
+            @Override
+            public boolean isExpressionLanguageSupported(final String propertyName) {
+                return false;
+            }
+
+            @Override
+            public String getProcessGroupIdentifier() {
+                return null;
+            }
+
+            @Override
+            public Collection<String> getReferencedParameters(final String propertyName) {
+                return null;
+            }
+
+            @Override
+            public boolean isParameterDefined(final String parameterName) {
+                return false;
+            }
+
+            @Override
+            public boolean isParameterSet(final String parameterName) {
+                return false;
+            }
+
+            @Override
+            public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+                return null;
+            }
+
+            @Override
+            public Map<String, String> getAllProperties() {
+                return null;
+            }
+        };
+
+        return abstractValidationContext.isDependencySatisfied(propertyDescriptor, propertyDescriptorLookup);
+    }
 }
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index c2db427..0ef2203 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -792,7 +792,19 @@ public class StandardProcessorTestRunner implements TestRunner {
         final Map<PropertyDescriptor, String> updatedProps = new HashMap<>(curProps);
 
         final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service);
-        final ValidationResult validationResult = property.validate(value, validationContext);
+        final boolean dependencySatisfied = validationContext.isDependencySatisfied(property, processor::getPropertyDescriptor);
+
+        final ValidationResult validationResult;
+        if (dependencySatisfied) {
+            validationResult = property.validate(value, validationContext);
+        } else {
+            validationResult = new ValidationResult.Builder()
+                .valid(true)
+                .input(value)
+                .subject(property.getDisplayName())
+                .explanation("Property is dependent upon another property, and this dependency is not satisfied, so value is considered valid")
+                .build();
+        }
 
         final String oldValue = updatedProps.get(property);
         updatedProps.put(property, value);
@@ -981,6 +993,7 @@ public class StandardProcessorTestRunner implements TestRunner {
         }
     }
 
+
     /**
      * Set the Run Schedule parameter (in milliseconds). If set, this will be the duration
      * between two calls of the onTrigger method.
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
index 23e5ebb..176cc98 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
@@ -90,7 +90,7 @@ public interface TestRunner {
      * <li>
      * If {@code initialize} is true, run all methods on the Processor that are
      * annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnScheduled @OnScheduled} annotation. If
+     * {@link org.apache.nifi.annotation.lifecycle.OnScheduled @OnScheduled} annotation. If
      * any of these methods throws an Exception, the Unit Test will fail.
      * </li>
      * <li>
@@ -105,7 +105,7 @@ public interface TestRunner {
      * As soon as the first thread finishes its execution of
      * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory) onTrigger},
      * all methods on the Processor that are annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnUnscheduled @OnUnscheduled} annotation
+     * {@link org.apache.nifi.annotation.lifecycle.OnUnscheduled @OnUnscheduled} annotation
      * are invoked. If any of these methods throws an Exception, the Unit Test
      * will fail.
      * </li>
@@ -115,13 +115,13 @@ public interface TestRunner {
      * <li>
      * If and only if the value of <code>shutdown</code> is true: Call all
      * methods on the Processor that is annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnStopped @OnStopped} annotation.
+     * {@link org.apache.nifi.annotation.lifecycle.OnStopped @OnStopped} annotation.
      * </li>
      * </ul>
      *
      * @param iterations number of iterations
      * @param stopOnFinish whether or not to run the Processor methods that are
-     *            annotated with {@link org.apache.nifi.processor.annotation.OnStopped @OnStopped}
+     *            annotated with {@link org.apache.nifi.annotation.lifecycle.OnStopped @OnStopped}
      * @param initialize true if must initialize
      */
     void run(int iterations, boolean stopOnFinish, final boolean initialize);
@@ -133,7 +133,7 @@ public interface TestRunner {
      * <li>
      * If {@code initialize} is true, run all methods on the Processor that are
      * annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnScheduled @OnScheduled} annotation. If
+     * {@link org.apache.nifi.annotation.lifecycle.OnScheduled @OnScheduled} annotation. If
      * any of these methods throws an Exception, the Unit Test will fail.
      * </li>
      * <li>
@@ -148,7 +148,7 @@ public interface TestRunner {
      * As soon as the first thread finishes its execution of
      * {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory) onTrigger},
      * all methods on the Processor that are annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnUnscheduled @OnUnscheduled} annotation
+     * {@link org.apache.nifi.annotation.lifecycle.OnUnscheduled @OnUnscheduled} annotation
      * are invoked. If any of these methods throws an Exception, the Unit Test
      * will fail.
      * </li>
@@ -158,22 +158,22 @@ public interface TestRunner {
      * <li>
      * If and only if the value of <code>shutdown</code> is true: Call all
      * methods on the Processor that is annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnStopped @OnStopped} annotation.
+     * {@link org.apache.nifi.annotation.lifecycle.OnStopped @OnStopped} annotation.
      * </li>
      * </ul>
      *
      * @param iterations number of iterations
      * @param stopOnFinish whether or not to run the Processor methods that are
-     *            annotated with {@link org.apache.nifi.processor.annotation.OnStopped @OnStopped}
+     *            annotated with {@link org.apache.nifi.annotation.lifecycle.OnStopped @OnStopped}
      * @param initialize true if must initialize
      * @param runWait indicates the amount of time in milliseconds that the framework should wait for
-     *            processors to stop running before calling the {@link org.apache.nifi.processor.annotation.OnUnscheduled @OnUnscheduled} annotation
+     *            processors to stop running before calling the {@link org.apache.nifi.annotation.lifecycle.OnUnscheduled @OnUnscheduled} annotation
      */
     void run(int iterations, boolean stopOnFinish, final boolean initialize, final long runWait);
 
     /**
      * Invokes all methods on the Processor that are annotated with the
-     * {@link org.apache.nifi.processor.annotation.OnShutdown @OnShutdown} annotation. If
+     * {@link org.apache.nifi.annotation.lifecycle.OnShutdown @OnShutdown} annotation. If
      * any of these methods throws an Exception, the Unit Test will fail
      */
     void shutdown();
@@ -507,7 +507,7 @@ public interface TestRunner {
      * considered "available", meaning that the queues of all Connections that
      * contain this Relationship are not full. This is generally used only when
      * dealing with Processors that use the
-     * {@link org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable}
+     * {@link org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable}
      * annotation.
      *
      * @param relationship to mark as available
@@ -519,7 +519,7 @@ public interface TestRunner {
      * name should be considered "available", meaning that the queues of all
      * Connections that contain this Relationship are not full. This is
      * generally used only when dealing with Processors that use the
-     * {@link org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable}
+     * {@link org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable}
      *
      * @param relationshipName relationship name
      */
@@ -530,7 +530,7 @@ public interface TestRunner {
      * considered "available", meaning that the queue of at least one Connection
      * that contain this Relationship is full. This is generally used only when
      * dealing with Processors that use the
-     * {@link org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable}
+     * {@link org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable}
      * annotation.
      *
      * @param relationship to mark as unavailable
@@ -542,7 +542,7 @@ public interface TestRunner {
      * should NOT be considered "available", meaning that the queue of at least
      * one Connection that contain this Relationship is full. This is generally
      * used only when dealing with Processors that use the
-     * {@link org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable}
+     * {@link org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable}
      *
      * @param relationshipName name of relationship.
      */
@@ -845,6 +845,7 @@ public interface TestRunner {
      */
     void setValidateExpressionUsage(boolean validate);
 
+
     /**
      * Removes the {@link PropertyDescriptor} from the {@link ProcessContext},
      * effectively setting its value to null, or the property's default value, if it has one.
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
index 567d860..7d10bac 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
@@ -84,16 +84,6 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
         .identifiesControllerService(RecordSchemaCacheService.class)
         .build();
 
-    static final PropertyDescriptor SCHEMA_PROTOCOL_VERSION = new Builder()
-            .name("schema-protocol-version")
-            .displayName("Schema Protocol Version")
-            .description("The protocol version to be used for Schema Write Strategies that require a protocol version, such as Hortonworks Schema Registry strategies. " +
-                    "Valid protocol versions for Hortonworks Schema Registry are integer values 1, 2, or 3.")
-            .required(false)
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .defaultValue("1")
-            .build();
 
     /**
      * This constant is just a base spec for the actual PropertyDescriptor.
@@ -106,18 +96,30 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
         .required(true)
         .build();
 
+    static final PropertyDescriptor SCHEMA_PROTOCOL_VERSION = new Builder()
+        .name("schema-protocol-version")
+        .displayName("Schema Protocol Version")
+        .description("The protocol version to be used for Schema Write Strategies that require a protocol version, such as Hortonworks Schema Registry strategies. " +
+            "Valid protocol versions for Hortonworks Schema Registry are integer values 1, 2, or 3.")
+        .required(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .dependsOn(SCHEMA_WRITE_STRATEGY, HWX_CONTENT_ENCODED_SCHEMA, HWX_SCHEMA_REF_ATTRIBUTES)
+        .defaultValue("1")
+        .build();
+
 
     private volatile ConfigurationContext configurationContext;
     private volatile SchemaAccessWriter schemaAccessWriter;
 
     private final List<AllowableValue> schemaWriteStrategyList = Collections.unmodifiableList(Arrays.asList(
-        SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA, CONFLUENT_ENCODED_SCHEMA, NO_SCHEMA));
+        NO_SCHEMA, SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA, CONFLUENT_ENCODED_SCHEMA));
 
     private final List<AllowableValue> schemaAccessStrategyList = Collections.unmodifiableList(Arrays.asList(
-        SCHEMA_NAME_PROPERTY, INHERIT_RECORD_SCHEMA, SCHEMA_TEXT_PROPERTY));
+        INHERIT_RECORD_SCHEMA, SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY));
 
     private final Set<String> schemaWriteStrategiesRequiringProtocolVersion = new HashSet<>(Arrays.asList(
-            HWX_CONTENT_ENCODED_SCHEMA.getValue(), HWX_SCHEMA_REF_ATTRIBUTES.getValue()));
+        HWX_CONTENT_ENCODED_SCHEMA.getValue(), HWX_SCHEMA_REF_ATTRIBUTES.getValue()));
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index 1b401f9..3d415e2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -32,6 +32,7 @@ import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDependency;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.documentation.DocumentationWriter;
@@ -555,6 +556,52 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
 
                     writeSimpleElement(xmlStreamWriter, "strong", text);
                 }
+
+                final Set<PropertyDependency> dependencies = property.getDependencies();
+                if (!dependencies.isEmpty()) {
+                    xmlStreamWriter.writeEmptyElement("br");
+                    xmlStreamWriter.writeEmptyElement("br");
+
+                    final boolean capitalizeThe;
+                    if (dependencies.size() == 1) {
+                        writeSimpleElement(xmlStreamWriter, "strong", "This Property is only considered if ");
+                        capitalizeThe = false;
+                    } else {
+                        writeSimpleElement(xmlStreamWriter, "strong", "This Property is only considered if all of the following conditions are met:");
+                        xmlStreamWriter.writeStartElement("ul");
+                        capitalizeThe = true;
+                    }
+
+                    for (final PropertyDependency dependency : dependencies) {
+                        final Set<String> dependentValues = dependency.getDependentValues();
+                        final String prefix = (capitalizeThe ? "The" : "the") + " <" + dependency.getPropertyName() + "> Property ";
+                        final String suffix;
+                        if (dependentValues == null) {
+                            suffix = "has a value specified.";
+                        } else if (dependentValues.size() == 1) {
+                            final String requiredValue = dependentValues.iterator().next();
+                            suffix = "has a value of \"" + requiredValue + "\".";
+                        } else {
+                            final StringBuilder sb = new StringBuilder("is set to one of the following values: ");
+
+                            for (final String dependentValue : dependentValues) {
+                                sb.append("\"").append(dependentValue).append("\", ");
+                            }
+
+                            // Delete the trailing ", "
+                            sb.setLength(sb.length() - 2);
+
+                            suffix = sb.toString();
+                        }
+
+                        writeSimpleElement(xmlStreamWriter, "strong", prefix + suffix);
+                    }
+
+                    if (dependencies.size() > 1) { // write </ul>
+                        xmlStreamWriter.writeEndElement();
+                    }
+                }
+
                 xmlStreamWriter.writeEndElement();
 
                 xmlStreamWriter.writeEndElement();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/pom.xml
new file mode 100644
index 0000000..cb167ec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/pom.xml
@@ -0,0 +1,41 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.13.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-framework-components</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/AbstractValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/AbstractValidationContext.java
new file mode 100644
index 0000000..9a00837
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/AbstractValidationContext.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi;
+
+import org.apache.nifi.components.PropertyDependency;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.controller.PropertyConfiguration;
+import org.apache.nifi.parameter.ParameterLookup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+public abstract class AbstractValidationContext implements ValidationContext {
+    private static final Logger logger = LoggerFactory.getLogger(AbstractValidationContext.class);
+
+    private final ParameterLookup parameterLookup;
+    private final Map<PropertyDescriptor, PropertyConfiguration> properties;
+
+    public AbstractValidationContext(final ParameterLookup parameterLookup, final Map<PropertyDescriptor, PropertyConfiguration> properties) {
+        this.parameterLookup = parameterLookup;
+        this.properties = properties;
+    }
+
+
+    public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup) {
+        return isDependencySatisfied(propertyDescriptor, propertyDescriptorLookup, new HashSet<>());
+    }
+
+    private boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup, final Set<String> propertiesSeen) {
+        final Set<PropertyDependency> dependencies = propertyDescriptor.getDependencies();
+        if (dependencies.isEmpty()) {
+            logger.debug("Dependency for {} is satisfied because it has no dependencies", propertyDescriptor);
+            return true;
+        }
+
+        final boolean added = propertiesSeen.add(propertyDescriptor.getName());
+        if (!added) {
+            logger.debug("Dependency for {} is not satisifed because its dependency chain contains a loop: {}", propertyDescriptor, propertiesSeen);
+            return false;
+        }
+
+        try {
+            for (final PropertyDependency dependency : dependencies) {
+                final String dependencyName = dependency.getPropertyName();
+
+                // Check if the property being depended upon has its dependencies satisfied.
+                final PropertyDescriptor dependencyDescriptor = propertyDescriptorLookup.apply(dependencyName);
+                if (dependencyDescriptor == null) {
+                    logger.debug("Dependency for {} is not satisfied because it has a dependency on {}, which has no property descriptor", propertyDescriptor, dependencyName);
+                    return false;
+                }
+
+                final PropertyConfiguration dependencyConfiguration = properties.get(dependencyDescriptor);
+                if (dependencyConfiguration == null) {
+                    logger.debug("Dependency for {} is not satisfied because it has a dependency on {}, which does not have a value", propertyDescriptor, dependencyName);
+                    return false;
+                }
+
+                final String dependencyValue = dependencyConfiguration.getEffectiveValue(parameterLookup);
+                if (dependencyValue == null) {
+                    logger.debug("Dependency for {} is not satisfied because it has a dependency on {}, which has a null value", propertyDescriptor, dependencyName);
+                    return false;
+                }
+
+                final boolean transitiveDependencySatisfied = isDependencySatisfied(dependencyDescriptor, propertyDescriptorLookup, propertiesSeen);
+                if (!transitiveDependencySatisfied) {
+                    logger.debug("Dependency for {} is not satisfied because it has a dependency on {} and {} does not have its dependencies satisfied",
+                        propertyDescriptor, dependencyName, dependencyName);
+                    return false;
+                }
+
+                // Check if the property being depended upon is set to one of the values that satisfies this dependency.
+                // If the dependency has no dependent values, then any non-null value satisfies the dependency.
+                // The value is already known to be non-null due to the check above.
+                final Set<String> dependentValues = dependency.getDependentValues();
+                if (dependentValues != null && !dependentValues.contains(dependencyValue)) {
+                    logger.debug("Dependency for {} is not satisfied because it depends on {}, which has a value of {}. Dependent values = {}",
+                        propertyDescriptor, dependencyName, dependencyValue, dependentValues);
+                    return false;
+                }
+            }
+
+            logger.debug("All dependencies for {} are satisfied", propertyDescriptor);
+
+            return true;
+        } finally {
+            propertiesSeen.remove(propertyDescriptor.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 0643418..024466d 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
@@ -20,6 +20,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
@@ -641,7 +642,15 @@ public abstract class AbstractComponentNode implements ComponentNode {
         final ParameterContext parameterContext = getParameterContext();
         final boolean assignedToProcessGroup = getProcessGroupIdentifier() != null;
 
+        final ConfigurableComponent component = getComponent();
         for (final PropertyDescriptor propertyDescriptor : validationContext.getProperties().keySet()) {
+            // If the property descriptor's dependency is not satisfied, the property does not need to be considered, as it's not relevant to the
+            // component's functionality.
+            final boolean dependencySatisfied = validationContext.isDependencySatisfied(propertyDescriptor, component::getPropertyDescriptor);
+            if (!dependencySatisfied) {
+                continue;
+            }
+
             final Collection<String> referencedParameters = validationContext.getReferencedParameters(propertyDescriptor.getName());
 
             if (parameterContext == null && !referencedParameters.isEmpty()) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 445b29b..86fcd42 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -45,6 +45,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-components</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
             <version>1.13.0-SNAPSHOT</version>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
index 3fc7e06..174eb93 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
@@ -17,6 +17,7 @@
  */
 package org.apache.nifi.processor;
 
+import org.apache.nifi.AbstractValidationContext;
 import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.Query.Range;
@@ -48,7 +49,7 @@ import java.util.Optional;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 
-public class StandardValidationContext implements ValidationContext {
+public class StandardValidationContext extends AbstractValidationContext implements ValidationContext {
 
     private final ControllerServiceProvider controllerServiceProvider;
     private final Map<PropertyDescriptor, PropertyConfiguration> properties;
@@ -70,6 +71,8 @@ public class StandardValidationContext implements ValidationContext {
             final String componentId,
             final VariableRegistry variableRegistry,
             final ParameterContext parameterContext) {
+        super(parameterContext, properties);
+
         this.controllerServiceProvider = controllerServiceProvider;
         this.properties = new HashMap<>(properties);
         this.annotationData = annotationData;
@@ -233,6 +236,7 @@ public class StandardValidationContext implements ValidationContext {
         return value != null;
     }
 
+
     @Override
     public String toString() {
         return "StandardValidationContext[componentId=" + componentId + ", properties=" + properties + "]";
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index 1293f48..0dd9cde 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -43,6 +43,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -644,6 +646,11 @@ public class TestStandardProcessorNode {
                     public boolean isParameterSet(final String parameterName) {
                         return false;
                     }
+
+                    @Override
+                    public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup) {
+                        return false;
+                    }
                 };
             }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
index 185e9d8..6b8feb1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/pom.xml
@@ -71,6 +71,11 @@
             <version>1.13.0-SNAPSHOT</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-components</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
             <groupId>com.google.code.gson</groupId>
             <artifactId>gson</artifactId>
             <version>2.8.2</version>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessProcessContext.java
index f9fcb9e..1de89bc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessProcessContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessProcessContext.java
@@ -123,6 +123,10 @@ public class StatelessProcessContext implements ProcessContext, ControllerServic
         return new StatelessPropertyValue(propValue, this.lookup, parameterContext, variableRegistry, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
     }
 
+    public PropertyDescriptor getPropertyDescriptor(final String propertyName) {
+        return component.getPropertyDescriptor(propertyName);
+    }
+
     @Override
     public PropertyValue newPropertyValue(final String rawValue) {
         return new StatelessPropertyValue(rawValue, this.lookup, parameterContext);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessValidationContext.java
index 9ed8434..c0aae1b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessValidationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessValidationContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.stateless.core;
 
+import org.apache.nifi.AbstractValidationContext;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -29,6 +30,8 @@ import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ParameterTokenList;
+import org.apache.nifi.parameter.StandardParameterTokenList;
 import org.apache.nifi.registry.VariableRegistry;
 
 import java.util.Collection;
@@ -40,7 +43,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
-public class StatelessValidationContext implements ValidationContext {
+public class StatelessValidationContext extends AbstractValidationContext implements ValidationContext {
 
     private final StatelessControllerServiceLookup lookup;
     private final Map<String, Boolean> expressionLanguageSupported;
@@ -51,6 +54,7 @@ public class StatelessValidationContext implements ValidationContext {
 
     public StatelessValidationContext(final StatelessProcessContext processContext, final StatelessControllerServiceLookup lookup, final StateManager stateManager,
                                       final VariableRegistry variableRegistry, final ParameterContext parameterContext) {
+        super(parameterContext, createPropertyConfigurationMap(processContext));
         this.processContext = processContext;
         this.lookup = lookup;
         this.stateManager = stateManager;
@@ -65,6 +69,20 @@ public class StatelessValidationContext implements ValidationContext {
         this.parameterContext = parameterContext;
     }
 
+    private static Map<PropertyDescriptor, PropertyConfiguration> createPropertyConfigurationMap(final StatelessProcessContext processContext) {
+        final Map<PropertyDescriptor, PropertyConfiguration> configurationMap = new HashMap<>();
+
+        for (final Map.Entry<String, String> entry : processContext.getAllProperties().entrySet()) {
+            final PropertyDescriptor descriptor = processContext.getPropertyDescriptor(entry.getKey());
+            final ParameterTokenList tokenList = new StandardParameterTokenList(entry.getValue(), Collections.emptyList());
+            final List<ParameterReference> parameterReferences = Collections.emptyList();
+            final PropertyConfiguration configuration = new PropertyConfiguration(entry.getValue(), tokenList, parameterReferences);
+            configurationMap.put(descriptor, configuration);
+        }
+
+        return configurationMap;
+    }
+
     @Override
     public PropertyValue newPropertyValue(final String rawValue) {
         return new StatelessPropertyValue(rawValue, this.lookup, parameterContext, variableRegistry);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index 1f6bce9..5e9d976 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -30,6 +30,7 @@
         <module>nifi-site-to-site</module>
         <module>nifi-repository-models</module>
         <module>nifi-flowfile-repo-serialization</module>
+        <module>nifi-framework-components</module>
         <module>nifi-framework-core</module>
         <module>nifi-framework-cluster-protocol</module>
         <module>nifi-framework-cluster</module>
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java
index eee7e37..9a8f49c 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java
@@ -25,6 +25,7 @@ import org.apache.nifi.expression.ExpressionLanguageCompiler;
 
 import java.util.Collection;
 import java.util.Map;
+import java.util.function.Function;
 
 public abstract class ValidationContextAdapter implements ValidationContext {
 
@@ -109,4 +110,9 @@ public abstract class ValidationContextAdapter implements ValidationContext {
     public Collection<String> getReferencedParameters(final String propertyName) {
         return innerValidationContext.getReferencedParameters(propertyName);
     }
+
+    @Override
+    public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function<String, PropertyDescriptor> propertyDescriptorLookup) {
+        return innerValidationContext.isDependencySatisfied(propertyDescriptor, propertyDescriptorLookup);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 13eb119..dfe60e1 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -390,7 +390,8 @@ public class MergeContent extends BinFiles {
         final Collection<ValidationResult> results = new ArrayList<>();
 
         final String delimiterStrategy = context.getProperty(DELIMITER_STRATEGY).getValue();
-        if(DELIMITER_STRATEGY_FILENAME.equals(delimiterStrategy)) {
+        final String mergeFormat = context.getProperty(MERGE_FORMAT).getValue();
+        if (DELIMITER_STRATEGY_FILENAME.getValue().equals(delimiterStrategy) && MERGE_FORMAT_CONCAT.getValue().equals(mergeFormat)) {
             final String headerValue = context.getProperty(HEADER).getValue();
             if (headerValue != null) {
                 results.add(StandardValidators.FILE_EXISTS_VALIDATOR.validate(HEADER.getName(), headerValue, context));
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
index 59190dd..a4c5fc0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
@@ -16,24 +16,6 @@
  */
 package org.apache.nifi.processors.standard;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.UnsupportedEncodingException;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.zip.ZipInputStream;
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.file.DataFileWriter;
@@ -59,6 +41,24 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.zip.ZipInputStream;
+
+import static org.junit.Assert.assertEquals;
+
 public class TestMergeContent {
 
     @BeforeClass
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
index dfcb41a..728f8d8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
@@ -59,7 +59,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class TestValidateRecord {
-
     private TestRunner runner;
 
     @Before
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
index 21ff5c6..dee9a4f 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
@@ -115,6 +115,7 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
             .defaultValue("1")
             .required(true)
             .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_GZIP)
             .build();
 
     private volatile boolean prettyPrint;
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
index d00799c..de84510 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java
@@ -48,6 +48,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.function.Supplier;
 
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
 import static org.apache.nifi.schema.inference.SchemaInferenceUtil.INFER_SCHEMA;
 import static org.apache.nifi.schema.inference.SchemaInferenceUtil.SCHEMA_CACHE;
 
@@ -69,7 +70,10 @@ public class JsonTreeReader extends SchemaRegistryService implements RecordReade
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
-        properties.add(SCHEMA_CACHE);
+        properties.add(new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_CACHE)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, INFER_SCHEMA)
+            .build());
         properties.add(DateTimeUtils.DATE_FORMAT);
         properties.add(DateTimeUtils.TIME_FORMAT);
         properties.add(DateTimeUtils.TIMESTAMP_FORMAT);
@@ -85,8 +89,9 @@ public class JsonTreeReader extends SchemaRegistryService implements RecordReade
 
     @Override
     protected List<AllowableValue> getSchemaAccessStrategyValues() {
-        final List<AllowableValue> allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues());
+        final List<AllowableValue> allowableValues = new ArrayList<>();
         allowableValues.add(INFER_SCHEMA);
+        allowableValues.addAll(super.getSchemaAccessStrategyValues());
         return allowableValues;
     }
 
diff --git a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/jetty/ControllerServiceTestContext.java b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/jetty/ControllerServiceTestContext.java
index 0f9cf2b..04cccdb 100644
--- a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/jetty/ControllerServiceTestContext.java
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/jetty/ControllerServiceTestContext.java
@@ -23,9 +23,12 @@ import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.util.MockControllerServiceInitializationContext;
 import org.apache.nifi.util.MockPropertyValue;
 
+import java.util.function.Function;
+
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -40,6 +43,7 @@ public class ControllerServiceTestContext {
         initializationContext = new MockControllerServiceInitializationContext(controllerService, id);
         doAnswer(invocation -> configurationContext.getProperty(invocation.getArgument(0)))
                 .when(validationContext).getProperty(any(PropertyDescriptor.class));
+        doReturn(true).when(validationContext).isDependencySatisfied(any(PropertyDescriptor.class), any(Function.class));
         controllerService.getPropertyDescriptors().forEach(prop -> setDefaultValue(prop));
     }
 
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
new file mode 100644
index 0000000..49c2dbb
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java
@@ -0,0 +1,118 @@
+/*
+ * 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.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+
+public class DependOnProperties extends AbstractProcessor {
+
+    static final AllowableValue FOO = new AllowableValue("foo");
+    static final AllowableValue BAR = new AllowableValue("bar");
+    static final AllowableValue BAZ = new AllowableValue("baz");
+
+    static final PropertyDescriptor ALWAYS_REQUIRED = new Builder()
+        .name("Always Required")
+        .displayName("Always Required")
+        .description("This property is always required")
+        .required(true)
+        .allowableValues(FOO, BAR, BAZ)
+        .build();
+
+    static final PropertyDescriptor ALWAYS_OPTIONAL = new Builder()
+        .name("Always Optional")
+        .displayName("Always Optional")
+        .description("This property is always optional")
+        .required(false)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .build();
+
+    static final PropertyDescriptor REQUIRED_IF_OPTIONAL_PROPERTY_SET = new Builder()
+        .name("Required If Optional Property Set")
+        .displayName("Required If Optional Property Set")
+        .description("This property is required if and only if the 'Always Optional' property is set (to anything)")
+        .required(true)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .dependsOn(ALWAYS_OPTIONAL)
+        .build();
+
+    static final PropertyDescriptor REQUIRED_IF_OPTIONAL_IS_FOO = new Builder()
+        .name("Required If Optional Property Set To Foo")
+        .displayName("Required If Optional Property Set To Foo")
+        .description("This property is required if and only if the 'Always Optional' property is set to the value 'foo'")
+        .required(true)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .dependsOn(ALWAYS_OPTIONAL, "foo")
+        .build();
+
+    static final PropertyDescriptor REQUIRED_IF_ALWAYS_REQUIRED_IS_BAR_OR_BAZ = new Builder()
+        .name("Required If Always Required Is Bar Or Baz")
+        .displayName("Required If Always Required Is Bar Or Baz")
+        .description("This property is required if and only if hte 'Always Required' property is set to the value 'bar' or the value 'baz'")
+        .required(true)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .dependsOn(ALWAYS_REQUIRED, BAR, BAZ)
+        .build();
+
+    static final PropertyDescriptor SECOND_LEVEL_DEPENDENCY = new Builder()
+        .name("Second Level Dependency")
+        .displayName("Second Level Dependency")
+        .description("Depends on 'Required If Optional Property Set To Foo'")
+        .required(false)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .dependsOn(REQUIRED_IF_OPTIONAL_IS_FOO)
+        .build();
+
+    static final PropertyDescriptor MULTIPLE_DEPENDENCIES = new Builder()
+        .name("Multiple Dependencies")
+        .displayName("Multiple Dependencies")
+        .description("Depends on Always Required = foo AND Always Optional = bar")
+        .required(true)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .dependsOn(ALWAYS_REQUIRED, FOO)
+        .dependsOn(ALWAYS_OPTIONAL, "bar")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(
+            ALWAYS_REQUIRED,
+            ALWAYS_OPTIONAL,
+            REQUIRED_IF_OPTIONAL_PROPERTY_SET,
+            REQUIRED_IF_OPTIONAL_IS_FOO,
+            REQUIRED_IF_ALWAYS_REQUIRED_IS_BAR_OR_BAZ,
+            SECOND_LEVEL_DEPENDENCY,
+            MULTIPLE_DEPENDENCIES);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+
+    }
+}
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 5170e95..7bea248 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
@@ -14,6 +14,7 @@
 # limitations under the License.
 
 org.apache.nifi.processors.tests.system.CountEvents
+org.apache.nifi.processors.tests.system.DependOnProperties
 org.apache.nifi.processors.tests.system.Duplicate
 org.apache.nifi.processors.tests.system.FakeProcessor
 org.apache.nifi.processors.tests.system.FakeDynamicPropertiesProcessor
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/validation/DependentPropertyValidationIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/validation/DependentPropertyValidationIT.java
new file mode 100644
index 0000000..8704de9
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/validation/DependentPropertyValidationIT.java
@@ -0,0 +1,99 @@
+/*
+ * 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.validation;
+
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class DependentPropertyValidationIT extends NiFiSystemIT {
+
+    @Test(timeout = 20_000)
+    public void testPropertyDependenciesAreValidatedProperly() throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity processor = getClientUtil().createProcessor("DependOnProperties");
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Required", "foo"));
+
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Processor will become invalid because "Always Optional" is set and "Required If Optional Property Set" is not.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Optional", "hello"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // Set the "Required If Optional Property Set" property. This should make the processor valid.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Required If Optional Property Set", "hello"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // If 'Always Optional' is set to 'foo', then the 'Required If Optional Property Set To Foo' property must be set.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Optional", "foo"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // Setting the 'Required If Optional Property Set To Foo' to any value will now make the processor valid.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Required If Optional Property Set To Foo", "hello"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Setting the 'Second Level Dependency' property to -42 will make the processor invalid because the property required a positive integer.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Second Level Dependency", "-42"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // If We now set the "Always Optional" property to 'other', then the 'Second Level Dependency' property should not be validated.
+        // This is because its dependency is not fully satisfied because of the 'transitive' dependency. I.e., it depends on 'Required If Optional Property Set To Foo'
+        // but since 'Required If Optional Property Set To Foo' is not available/validated (because Always Optional is set to 'other'), then 'Second Level Dependency' is also not available/validated.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Optional", "other"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Setting the 'Always Required' property to 'bar' will result in requiring the 'Required If Always Required Is Bar Or Baz' property to be set.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Required", "bar"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // Setting the 'Required If Always Required Is Bar Or Baz' to any positive integer should now make the processor valid.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Required If Always Required Is Bar Or Baz", "42"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Setting the property to -42 should result in an invalid processor because the property must be a positive integer.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Required If Always Required Is Bar Or Baz", "-42"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // Changing the 'Always Required' back to 'foo' should result in the processor becoming valid again. Even though 'Required If Always Required Is Bar Or Baz' is invalid,
+        // the processor is still valid because this property will no longer be considered.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Required", "foo"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Setting Always Required = foo and Always Optional = bar means that "Multiple Dependencies" will be required.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Always Optional", "bar"));
+        getClientUtil().waitForInvalidProcessor(processor.getId());
+
+        // Setting the "Multiple Dependencies" property to anything other than empty string will result in the processor becoming valid.
+        getClientUtil().updateProcessorProperties(processor, Collections.singletonMap("Multiple Dependencies", "bar"));
+        getClientUtil().waitForValidProcessor(processor.getId());
+
+        // Processor should stay valid if we make Multiple Dependencies empty string (invalid) but also change Always Optional to "other" because that will result in "Multiple Dependencies"
+        // no longer being available/validated.
+        final Map<String, String> props = new HashMap<>();
+        props.put("Multiple Dependencies", "");
+        props.put("Always Optional", "other");
+        getClientUtil().updateProcessorProperties(processor, props);
+        getClientUtil().waitForValidProcessor(processor.getId());
+    }
+
+}


[nifi] 07/07: NIFI-1121 Fix Schema Name and Schema Branch properties

Posted by bb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d773521ee01f3a34fdd21365a86c8de7b82077a6
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Tue Nov 3 15:33:26 2020 -0500

    NIFI-1121 Fix Schema Name and Schema Branch properties
    
    This closes #4512.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
index 31612e6..de8111f 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/schema/access/SchemaAccessUtils.java
@@ -88,7 +88,7 @@ public class SchemaAccessUtils {
                     "If the chosen Schema Registry does not support branching, this value will be ignored.")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .dependsOn(SCHEMA_ACCESS_STRATEGY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, SCHEMA_NAME_PROPERTY)
             .required(false)
             .build();
 
@@ -99,7 +99,7 @@ public class SchemaAccessUtils {
                     "If not specified then the latest version of the schema will be retrieved.")
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .dependsOn(SCHEMA_ACCESS_STRATEGY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)
+            .dependsOn(SCHEMA_ACCESS_STRATEGY, SCHEMA_NAME_PROPERTY)
             .required(false)
             .build();