You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/06/07 12:05:11 UTC

[GitHub] [nifi] gresockj opened a new pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

gresockj opened a new pull request #5131:
URL: https://github.com/apache/nifi/pull/5131


   <!--
     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.
   -->
   #### Description of PR
   
   Refactors NiFiProperties and NiFiRegistryProperties:
   * NiFiProperties and NiFiRegistryProperties now implement a common interface, `ApplicationProperties`.
   * ProtectedNiFiProperties and ProtectedNiFiRegistryProperties now implement a common interface, `ProtectedApplicationProperties`.
   * ApplicationPropertiesProtector now handles the majority of the protection logic for ProtectedNiFiProperties and ProtectedNiFiRegistryProperties, so that sensitive property handling will be uniform between NiFi and NiFi Registry
   
   Refactors SensitivePropertyProviderFactory:
   * There is now only one SensitivePropertyProviderFactory, which can be configured with a master key and a new BootstrapProperties object, which represents the bootstrap.conf file.  All configuration for future SensitivePropertiesProviders (SPPs) is intended to go into bootstrap.conf.
   * The single StandardSensitivePropertyProviderFactory implementation is intended to select the appropriate SPP based on a new SensitivePropertyProtectionScheme enum.
   * Each SPP implementation can choose to use the master key and/or the BootstrapProperties provided in the SPP Factory
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on JDK 8?
   - [x] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648559487



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProtector.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.properties;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Encapsulates methods needed to protect application properties.
+ * @param <T> The ProtectedApplicationProperties type
+ * @param <U> The ApplicationProperties type
+ */
+public interface SensitivePropertyProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties> {
+
+    /**
+     * Returns the number of properties, excluding protection scheme properties.
+     * <p>
+     * Example:
+     * <p>
+     * key: E(value, key)
+     * key.protected: aes/gcm/256
+     * key2: value2
+     * <p>
+     * would return size 2
+     *
+     * @return the count of real properties
+     */
+    int size();
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the complete set of property keys, including any protection keys (i.e. 'x.y.z.protected').
+     *
+     * @return the set of property keys
+     */
+    Set<String> getPropertyKeysIncludingProtectionSchemes();
+
+    /**
+     * Returns a list of the keys identifying "sensitive" properties. There is a default list,
+     * and additional keys can be provided in the {@code nifi.sensitive.props.additional.keys} property in the ApplicationProperties.
+     *
+     * @return the list of sensitive property keys
+     */
+    List<String> getSensitivePropertyKeys();
+
+    /**
+     * Returns a list of the keys identifying "sensitive" properties. There is a default list,
+     * and additional keys can be provided in the {@code nifi.sensitive.props.additional.keys} property in the ApplicationProperties.
+     *
+     * @return the list of sensitive property keys
+     */
+    List<String> getPopulatedSensitivePropertyKeys();
+
+    /**
+     * Returns true if any sensitive keys are protected.
+     *
+     * @return true if any key is protected; false otherwise
+     */
+    boolean hasProtectedKeys();
+
+    /**
+     * Returns the unique set of all protection schemes currently in use for this instance.
+     *
+     * @return the set of protection schemes
+     */
+    Set<String> getProtectionSchemes();
+
+    /**
+     * Returns a Map of the keys identifying "sensitive" properties that are currently protected and the "protection" key for each.
+     * This may or may not include all properties marked as sensitive.
+     *
+     * @return the Map of protected property keys and the protection identifier for each
+     */
+    Map<String, String> getProtectedPropertyKeys();
+
+    /**
+     * Returns the local provider cache (null-safe) as a Map of protection schemes -> implementations.
+     *
+     * @return the map
+     */
+    Map<String, SensitivePropertyProvider> getSensitivePropertyProviders();
+
+    /**
+     * Returns a percentage of the total number of populated properties marked as sensitive that are currently protected.
+     *
+     * @return the percent of sensitive properties marked as protected
+     */
+    int getPercentOfSensitivePropertiesProtected();

Review comment:
       Actually, I only see it used in unit tests, so I'll refactor the implementation into the tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] thenatog commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
thenatog commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r650008806



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)

Review comment:
       Are these examples here for a reason?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648572018



##########
File path: nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
##########
@@ -30,8 +30,8 @@
             <version>1.14.0-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>

Review comment:
       Yes, since the CryptoKeyLoader (in nifi-registry-properties) is now used.   Since nifi-registry-properties depends on nifi-properties, I didn't include them both.  However, I agree it would be more clear to include both, so I'll add nifi-properties again.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648372410



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();
+
+    /**
+     * Returns the application properties in a basic Properties object.
+     * @return The basic Properties object
+     */
+    Properties toBasicProperties();

Review comment:
       Fair enough, removing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648568600



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
##########
@@ -87,12 +84,19 @@ private static JAXBContext initializeJaxbContext() {
     }
 
     private Authorizer authorizer;
-    private NiFiProperties properties;
     private ExtensionManager extensionManager;
     private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
     private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
     private final Map<String, Authorizer> authorizers = new HashMap<>();
 
+    public NiFiProperties getProperties() {
+        return properties;
+    }

Review comment:
       I think only for Spring dependency injection purposes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648524520



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.properties;
+
+import java.nio.file.Path;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+public class BootstrapProperties {
+    private static final String PROPERTY_KEY_FORMAT = "%s.%s";
+    private static final String BOOTSTRAP_SENSITIVE_KEY = "bootstrap.sensitive.key";
+
+    private final String propertyPrefix;
+    private final Properties properties;
+    private final Path configFilePath;
+
+    public BootstrapProperties(final String propertyPrefix, final Properties properties, final Path configFilePath) {
+        Objects.requireNonNull(propertyPrefix, "Property prefix is required");
+        Objects.requireNonNull(properties, "Properties are required");
+        this.propertyPrefix = propertyPrefix;
+        this.configFilePath = configFilePath;
+        this.properties = filterProperties(properties);
+    }
+
+    /**
+     * Returns the path to the bootstrap config file.
+     * @return The path to the file
+     */
+    public Path getConfigFilePath() {
+        return configFilePath;
+    }
+
+    /**
+     * Includes only the properties starting with the propertyPrefix.
+     * @param properties Unfiltered properties
+     * @return The filtered properties
+     */
+    private Properties filterProperties(final Properties properties) {
+        final Properties filteredProperties = new Properties();
+        for(final Enumeration<Object> e = properties.keys() ; e.hasMoreElements(); ) {
+            final String key = e.nextElement().toString();
+            if (key.startsWith(propertyPrefix)) {
+                filteredProperties.put(key, properties.getProperty(key));
+            }
+        }
+        return filteredProperties;
+    }
+
+    private String getPropertyKey(final String subKey) {
+        return String.format(PROPERTY_KEY_FORMAT, propertyPrefix, subKey);
+    }
+
+    /**
+     * Retrieves the value of the property by the full property key.
+     * @param key The property key in the bootstrap configuration
+     * @return The value of the property
+     */
+    public String getProperty(final String key) {
+        return properties.getProperty(key);
+    }
+
+    /**
+     * Returns the bootstrap sensitive key.
+     * @return The bootstrap sensitive key
+     */
+    public Optional<String> getBootstrapSensitiveKey() {

Review comment:
       Since the key is used as a hex string in quite a few places in the code already, I wanted to keep it as a String here.  I like this idea, but perhaps as a future refactoring.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648363858



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();
+
+    /**
+     * Returns the application properties in a basic Properties object.
+     * @return The basic Properties object
+     */
+    Properties toBasicProperties();
+
+    // Following are specific properties expected for all applications
+
+    /**
+     * Keystore path for TLS configuration
+     * @return The keystore path
+     */
+    String getKeyStorePath();
+
+    /**
+     * Keystore type for TLS configuration
+     * @return The keystore type
+     */
+    String getKeyStoreType();
+
+    /**
+     * Keystore password for TLS configuration
+     * @return The keystore password
+     */
+    String getKeyStorePassword();
+
+    /**
+     * Key password for TLS configuration
+     * @return The key password
+     */
+    String getKeyPassword();
+
+    /**
+     * Truststore path for TLS configuration
+     * @return The truststore path
+     */
+    String getTrustStorePath();
+
+    /**
+     * Truststore type for TLS configuration
+     * @return The truststore type
+     */
+    String getTrustStoreType();
+
+    /**
+     * Truststore password for TLS configuration
+     * @return The truststore password
+     */
+    String getTrustStorePassword();

Review comment:
       Good point, now that we're using boostrap.conf instead of nifi.properties to configure the Sensitive Properties Providers, these methods are not needed with this PR.  I'll pull them out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] thenatog commented on pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
thenatog commented on pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#issuecomment-859713906


   I just tested out running NiFi with a secure cluster and encrypting the configs/flow.xml to make sure everything was still operating functionally. The code and changes looks good to me. +1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648572018



##########
File path: nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
##########
@@ -30,8 +30,8 @@
             <version>1.14.0-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>

Review comment:
       Yes, since the CryptoKeyUtil (in nifi-registry-properties) is now used.   Since nifi-registry-properties depends on nifi-properties, I didn't include them both.  However, I agree it would be more clear to include both, so I'll add nifi-properties again.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648451941



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.properties;
+
+import java.nio.file.Path;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+public class BootstrapProperties {

Review comment:
       Hmm, yeah I see what you mean with the similar properties.  I'll go ahead and implement the interface and add a class comment to reduce confusion.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] thenatog commented on pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
thenatog commented on pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#issuecomment-856914804


   Reviewing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r650015205



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)

Review comment:
       It was just pulled over from the existing code, but I removed the comments in 5dbad70.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r649598985



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ImmutableProperties.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.properties;
+
+import java.util.Set;
+
+/**
+ * A base interface for providing an immutable set of properties.
+ */
+public interface ImmutableProperties {

Review comment:
       The name would seem to preclude implementations that are mutable, however, that is not necessarily enforceable.  What do you think about naming this `ReadableProperties`?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/main/java/org/apache/nifi/properties/ProtectedNiFiProperties.java
##########
@@ -16,35 +16,33 @@
  */
 package org.apache.nifi.properties;
 
-import static java.util.Arrays.asList;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import java.util.stream.Collectors;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import static java.util.Arrays.asList;
 
 /**
  * Decorator class for intermediate phase when {@link NiFiPropertiesLoader} loads the
  * raw properties file and performs unprotection activities before returning a clean
- * implementation of {@link NiFiProperties}, likely {@link StandardNiFiProperties}.
+ * implementation of {@link NiFiProperties}.
  * This encapsulates the sensitive property access logic from external consumers
  * of {@code NiFiProperties}.
  */
-class ProtectedNiFiProperties extends StandardNiFiProperties {
+class ProtectedNiFiProperties extends NiFiProperties implements ProtectedProperties<NiFiProperties>,
+        SensitivePropertyProtector<ProtectedNiFiProperties, NiFiProperties> {
     private static final Logger logger = LoggerFactory.getLogger(ProtectedNiFiProperties.class);
 
-    private NiFiProperties niFiProperties;
+    private SensitivePropertyProtector<ProtectedNiFiProperties, NiFiProperties> propertyProtectionDelegate;
 
-    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+    private NiFiProperties underlyingProperties;

Review comment:
       Recommend renaming to applicationProperties given method renaming.

##########
File path: nifi-commons/nifi-sensitive-property-provider/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0"?>
+<!--
+  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.
+-->
+<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">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-commons</artifactId>
+        <version>1.14.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-sensitive-property-provider</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>1.14.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.bouncycastle</groupId>
+            <artifactId>bcprov-jdk15on</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>

Review comment:
       Is there a class that uses JUL logging in this module, or is this dependency unnecessary?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties",

Review comment:
       Recommend streamlining the log message and changing it to debug
   ```suggestion
               logger.debug("Protected Properties [{}] Sensitive Properties [{}]",
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(PROTECTED_KEY_SUFFIX)) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedProperties.getApplicationProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }

Review comment:
       This could be simplified using Objects.requireNonNull()

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));

Review comment:
       Recommend removing this comment or perhaps adding a TODO to change it later, but it otherwise seems unnecessary.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    public static final String PROTECTED_KEY_SUFFIX = ".protected";
+
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedProperties}.
+     *
+     * @param protectedProperties the ProtectedProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedProperties) {
+        this.protectedProperties = protectedProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + PROTECTED_KEY_SUFFIX;
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(PROTECTED_KEY_SUFFIX));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedProperties.getApplicationProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedProperties.getApplicationProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }

Review comment:
       Recommend removing this comment since Groovy is only used in test classes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648548315



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(".protected"));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedApplicationProperties.getUnderlyingProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedApplicationProperties.getUnderlyingProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedApplicationProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedApplicationProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public int getPercentOfSensitivePropertiesProtected() {
+        return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties ({}%)",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size(),
+                    getPercentOfSensitivePropertiesProtected());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(".protected")) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedApplicationProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedApplicationProperties.getUnderlyingProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }
+
+        if (getSensitivePropertyProviders().containsKey(sensitivePropertyProvider.getIdentifierKey())) {
+            throw new UnsupportedOperationException("Cannot overwrite existing sensitive property provider registered for " + sensitivePropertyProvider.getIdentifierKey());
+        }
+
+        getSensitivePropertyProviders().put(sensitivePropertyProvider.getIdentifierKey(), sensitivePropertyProvider);
+    }
+
+    @Override
+    public String toString() {
+        final Set<String> providers = getSensitivePropertyProviders().keySet();
+        return new StringBuilder("ApplicationPropertiesProtector instance with ")
+                .append(size()).append(" properties (")
+                .append(getProtectedPropertyKeys().size())
+                .append(" protected) and ")
+                .append(providers.size())
+                .append(" sensitive property providers: ")
+                .append(StringUtils.join(providers, ", "))
+                .toString();
+    }
+
+    @Override
+    public Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+        if (localProviderCache == null) {
+            localProviderCache = new HashMap<>();
+        }
+
+        return localProviderCache;
+    }
+
+    private SensitivePropertyProvider getSensitivePropertyProvider(final String protectionScheme) {
+        if (isProviderAvailable(protectionScheme)) {
+            return getSensitivePropertyProviders().get(protectionScheme);
+        } else {
+            throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+        }
+    }
+
+    private boolean isProviderAvailable(final String protectionScheme) {
+        return getSensitivePropertyProviders().containsKey(protectionScheme);
+    }
+
+    /**
+     * If the value is protected, unprotects it and returns it. If not, returns the original value.
+     *
+     * @param key            the retrieved property key
+     * @param retrievedValue the retrieved property value
+     * @return the unprotected value
+     */
+    private String unprotectValue(final String key, final String retrievedValue) {
+        // Checks if the key is sensitive and marked as protected
+        if (isPropertyProtected(key)) {
+            final String protectionScheme = getProperty(getProtectionKey(key));
+
+            // No provider registered for this scheme, so just return the value
+            if (!isProviderAvailable(protectionScheme)) {
+                logger.warn("No provider available for {} so passing the protected {} value back", protectionScheme, key);
+                return retrievedValue;
+            }
+
+            try {
+                final SensitivePropertyProvider sensitivePropertyProvider = getSensitivePropertyProvider(protectionScheme);
+                return sensitivePropertyProvider.unprotect(retrievedValue);
+            } catch (SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error unprotecting value for " + key, e.getCause());

Review comment:
       Ha, good question.  I don't see a reason, and I don't even think a subclassed exception would be necessary.  I'll just let the enclosing call throw the exception.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648548315



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(".protected"));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedApplicationProperties.getUnderlyingProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedApplicationProperties.getUnderlyingProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedApplicationProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedApplicationProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public int getPercentOfSensitivePropertiesProtected() {
+        return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties ({}%)",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size(),
+                    getPercentOfSensitivePropertiesProtected());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(".protected")) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedApplicationProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedApplicationProperties.getUnderlyingProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }
+
+        if (getSensitivePropertyProviders().containsKey(sensitivePropertyProvider.getIdentifierKey())) {
+            throw new UnsupportedOperationException("Cannot overwrite existing sensitive property provider registered for " + sensitivePropertyProvider.getIdentifierKey());
+        }
+
+        getSensitivePropertyProviders().put(sensitivePropertyProvider.getIdentifierKey(), sensitivePropertyProvider);
+    }
+
+    @Override
+    public String toString() {
+        final Set<String> providers = getSensitivePropertyProviders().keySet();
+        return new StringBuilder("ApplicationPropertiesProtector instance with ")
+                .append(size()).append(" properties (")
+                .append(getProtectedPropertyKeys().size())
+                .append(" protected) and ")
+                .append(providers.size())
+                .append(" sensitive property providers: ")
+                .append(StringUtils.join(providers, ", "))
+                .toString();
+    }
+
+    @Override
+    public Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+        if (localProviderCache == null) {
+            localProviderCache = new HashMap<>();
+        }
+
+        return localProviderCache;
+    }
+
+    private SensitivePropertyProvider getSensitivePropertyProvider(final String protectionScheme) {
+        if (isProviderAvailable(protectionScheme)) {
+            return getSensitivePropertyProviders().get(protectionScheme);
+        } else {
+            throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+        }
+    }
+
+    private boolean isProviderAvailable(final String protectionScheme) {
+        return getSensitivePropertyProviders().containsKey(protectionScheme);
+    }
+
+    /**
+     * If the value is protected, unprotects it and returns it. If not, returns the original value.
+     *
+     * @param key            the retrieved property key
+     * @param retrievedValue the retrieved property value
+     * @return the unprotected value
+     */
+    private String unprotectValue(final String key, final String retrievedValue) {
+        // Checks if the key is sensitive and marked as protected
+        if (isPropertyProtected(key)) {
+            final String protectionScheme = getProperty(getProtectionKey(key));
+
+            // No provider registered for this scheme, so just return the value
+            if (!isProviderAvailable(protectionScheme)) {
+                logger.warn("No provider available for {} so passing the protected {} value back", protectionScheme, key);
+                return retrievedValue;
+            }
+
+            try {
+                final SensitivePropertyProvider sensitivePropertyProvider = getSensitivePropertyProvider(protectionScheme);
+                return sensitivePropertyProvider.unprotect(retrievedValue);
+            } catch (SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error unprotecting value for " + key, e.getCause());

Review comment:
       Edit: actually, I see now that the spp.unprotect() method doesn't have access to the key, and this code appears to want to include the key in the exception, which could be valuable.  Perhaps instead a logger.error noting the key, before rethrowing the exception?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5131:
URL: https://github.com/apache/nifi/pull/5131


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648544417



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(".protected"));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedApplicationProperties.getUnderlyingProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedApplicationProperties.getUnderlyingProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedApplicationProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedApplicationProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public int getPercentOfSensitivePropertiesProtected() {
+        return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties ({}%)",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size(),
+                    getPercentOfSensitivePropertiesProtected());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(".protected")) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedApplicationProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedApplicationProperties.getUnderlyingProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }
+
+        if (getSensitivePropertyProviders().containsKey(sensitivePropertyProvider.getIdentifierKey())) {
+            throw new UnsupportedOperationException("Cannot overwrite existing sensitive property provider registered for " + sensitivePropertyProvider.getIdentifierKey());
+        }
+
+        getSensitivePropertyProviders().put(sensitivePropertyProvider.getIdentifierKey(), sensitivePropertyProvider);
+    }
+
+    @Override
+    public String toString() {
+        final Set<String> providers = getSensitivePropertyProviders().keySet();
+        return new StringBuilder("ApplicationPropertiesProtector instance with ")
+                .append(size()).append(" properties (")
+                .append(getProtectedPropertyKeys().size())
+                .append(" protected) and ")
+                .append(providers.size())
+                .append(" sensitive property providers: ")
+                .append(StringUtils.join(providers, ", "))
+                .toString();
+    }
+
+    @Override
+    public Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+        if (localProviderCache == null) {
+            localProviderCache = new HashMap<>();
+        }
+
+        return localProviderCache;
+    }
+
+    private SensitivePropertyProvider getSensitivePropertyProvider(final String protectionScheme) {
+        if (isProviderAvailable(protectionScheme)) {
+            return getSensitivePropertyProviders().get(protectionScheme);
+        } else {
+            throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+        }
+    }
+
+    private boolean isProviderAvailable(final String protectionScheme) {
+        return getSensitivePropertyProviders().containsKey(protectionScheme);
+    }
+
+    /**
+     * If the value is protected, unprotects it and returns it. If not, returns the original value.
+     *
+     * @param key            the retrieved property key
+     * @param retrievedValue the retrieved property value
+     * @return the unprotected value
+     */
+    private String unprotectValue(final String key, final String retrievedValue) {
+        // Checks if the key is sensitive and marked as protected
+        if (isPropertyProtected(key)) {
+            final String protectionScheme = getProperty(getProtectionKey(key));
+
+            // No provider registered for this scheme, so just return the value
+            if (!isProviderAvailable(protectionScheme)) {
+                logger.warn("No provider available for {} so passing the protected {} value back", protectionScheme, key);

Review comment:
       Agreed.  This was directly from the previous code, but I'll take the opportunity to throw IllegalStateException.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648267458



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();
+
+    /**
+     * Returns the application properties in a basic Properties object.
+     * @return The basic Properties object
+     */
+    Properties toBasicProperties();
+
+    // Following are specific properties expected for all applications
+
+    /**
+     * Keystore path for TLS configuration
+     * @return The keystore path
+     */
+    String getKeyStorePath();
+
+    /**
+     * Keystore type for TLS configuration
+     * @return The keystore type
+     */
+    String getKeyStoreType();
+
+    /**
+     * Keystore password for TLS configuration
+     * @return The keystore password
+     */
+    String getKeyStorePassword();
+
+    /**
+     * Key password for TLS configuration
+     * @return The key password
+     */
+    String getKeyPassword();
+
+    /**
+     * Truststore path for TLS configuration
+     * @return The truststore path
+     */
+    String getTrustStorePath();
+
+    /**
+     * Truststore type for TLS configuration
+     * @return The truststore type
+     */
+    String getTrustStoreType();
+
+    /**
+     * Truststore password for TLS configuration
+     * @return The truststore password
+     */
+    String getTrustStorePassword();

Review comment:
       Although these methods are certainly convenient, I'm not sure about including them at the top-level interface, particularly since they can return null depending on the configuration.  If this is something we should consider, I think it would be better as a separate issue, and perhaps as an extension of this interface when the properties are not null.

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();
+
+    /**
+     * Returns the application properties in a basic Properties object.
+     * @return The basic Properties object
+     */
+    Properties toBasicProperties();

Review comment:
       This is also a useful convenience, but in terms of interface definition, particularly when it comes to unit tests, it seems better to separate out somewhere else.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";

Review comment:
       Recommend creating a static variable named something like `PROTECTED_SUFFIX` for `.protected` and reusing across methods.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(".protected"));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedApplicationProperties.getUnderlyingProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedApplicationProperties.getUnderlyingProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedApplicationProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedApplicationProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public int getPercentOfSensitivePropertiesProtected() {
+        return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties ({}%)",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size(),
+                    getPercentOfSensitivePropertiesProtected());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(".protected")) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedApplicationProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedApplicationProperties.getUnderlyingProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }
+
+        if (getSensitivePropertyProviders().containsKey(sensitivePropertyProvider.getIdentifierKey())) {
+            throw new UnsupportedOperationException("Cannot overwrite existing sensitive property provider registered for " + sensitivePropertyProvider.getIdentifierKey());
+        }
+
+        getSensitivePropertyProviders().put(sensitivePropertyProvider.getIdentifierKey(), sensitivePropertyProvider);
+    }
+
+    @Override
+    public String toString() {
+        final Set<String> providers = getSensitivePropertyProviders().keySet();
+        return new StringBuilder("ApplicationPropertiesProtector instance with ")
+                .append(size()).append(" properties (")
+                .append(getProtectedPropertyKeys().size())
+                .append(" protected) and ")
+                .append(providers.size())
+                .append(" sensitive property providers: ")
+                .append(StringUtils.join(providers, ", "))
+                .toString();
+    }
+
+    @Override
+    public Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+        if (localProviderCache == null) {
+            localProviderCache = new HashMap<>();
+        }
+
+        return localProviderCache;
+    }
+
+    private SensitivePropertyProvider getSensitivePropertyProvider(final String protectionScheme) {
+        if (isProviderAvailable(protectionScheme)) {
+            return getSensitivePropertyProviders().get(protectionScheme);
+        } else {
+            throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+        }
+    }
+
+    private boolean isProviderAvailable(final String protectionScheme) {
+        return getSensitivePropertyProviders().containsKey(protectionScheme);
+    }
+
+    /**
+     * If the value is protected, unprotects it and returns it. If not, returns the original value.
+     *
+     * @param key            the retrieved property key
+     * @param retrievedValue the retrieved property value
+     * @return the unprotected value
+     */
+    private String unprotectValue(final String key, final String retrievedValue) {
+        // Checks if the key is sensitive and marked as protected
+        if (isPropertyProtected(key)) {
+            final String protectionScheme = getProperty(getProtectionKey(key));
+
+            // No provider registered for this scheme, so just return the value
+            if (!isProviderAvailable(protectionScheme)) {
+                logger.warn("No provider available for {} so passing the protected {} value back", protectionScheme, key);
+                return retrievedValue;
+            }
+
+            try {
+                final SensitivePropertyProvider sensitivePropertyProvider = getSensitivePropertyProvider(protectionScheme);
+                return sensitivePropertyProvider.unprotect(retrievedValue);
+            } catch (SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error unprotecting value for " + key, e.getCause());

Review comment:
       Is the reason for catching and rethrowing the Exception in order to include the property key?  It seems confusing initially, perhaps a subclassed exception would clarify what is happening?

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();

Review comment:
       Is this method necessary at this interface level?

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedApplicationProperties.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.properties;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Represents a protected set of ApplicationProperties, with methods regarding which sensitive properties
+ * are protected.
+ * @param <T> The ApplicationProperties type
+ */
+public interface ProtectedApplicationProperties<T extends ApplicationProperties> {
+
+    /**
+     * Additional sensitive properties keys
+     * @return Additional sensitive properties keys
+     */
+    String getAdditionalSensitivePropertiesKeys();
+
+    /**
+     * Returns the name of the property that specifies the additional sensitive properties keys
+     * @return Name of additional sensitive properties keys
+     */
+    String getAdditionalSensitivePropertiesKeysName();
+
+    /**
+     * Additional sensitive properties keys
+     * @return Additional sensitive properties keys
+     */
+    List<String> getDefaultSensitiveProperties();
+
+    /**
+     * Returns the underlying application properties.
+     * @return The underlying properties
+     */
+    T getUnderlyingProperties();

Review comment:
       Renaming the interface would provide the opportunity to rename this to `getApplicationProperties()`,

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationPropertiesBase.java
##########
@@ -18,64 +18,52 @@
 
 import java.util.Enumeration;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import org.apache.nifi.util.NiFiProperties;
 
-public class StandardNiFiProperties extends NiFiProperties {
+public abstract class ApplicationPropertiesBase implements ApplicationProperties {

Review comment:
       Although there are different naming conventions used in various places, what do you think about naming this AbstractApplicationProperties?  Alternatively, if the keystore and truststore methods are removed from the interface definition, perhaps this does not need to be `abstract` and could instead be named `StandardApplicationProperties`.

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.properties;
+
+import java.nio.file.Path;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+public class BootstrapProperties {
+    private static final String PROPERTY_KEY_FORMAT = "%s.%s";
+    private static final String BOOTSTRAP_SENSITIVE_KEY = "bootstrap.sensitive.key";
+
+    private final String propertyPrefix;
+    private final Properties properties;
+    private final Path configFilePath;
+
+    public BootstrapProperties(final String propertyPrefix, final Properties properties, final Path configFilePath) {
+        Objects.requireNonNull(propertyPrefix, "Property prefix is required");
+        Objects.requireNonNull(properties, "Properties are required");
+        this.propertyPrefix = propertyPrefix;
+        this.configFilePath = configFilePath;
+        this.properties = filterProperties(properties);
+    }
+
+    /**
+     * Returns the path to the bootstrap config file.
+     * @return The path to the file
+     */
+    public Path getConfigFilePath() {
+        return configFilePath;
+    }
+
+    /**
+     * Includes only the properties starting with the propertyPrefix.
+     * @param properties Unfiltered properties
+     * @return The filtered properties
+     */
+    private Properties filterProperties(final Properties properties) {
+        final Properties filteredProperties = new Properties();
+        for(final Enumeration<Object> e = properties.keys() ; e.hasMoreElements(); ) {
+            final String key = e.nextElement().toString();
+            if (key.startsWith(propertyPrefix)) {
+                filteredProperties.put(key, properties.getProperty(key));
+            }
+        }
+        return filteredProperties;
+    }
+
+    private String getPropertyKey(final String subKey) {
+        return String.format(PROPERTY_KEY_FORMAT, propertyPrefix, subKey);
+    }
+
+    /**
+     * Retrieves the value of the property by the full property key.
+     * @param key The property key in the bootstrap configuration
+     * @return The value of the property
+     */
+    public String getProperty(final String key) {
+        return properties.getProperty(key);
+    }
+
+    /**
+     * Returns the bootstrap sensitive key.
+     * @return The bootstrap sensitive key
+     */
+    public Optional<String> getBootstrapSensitiveKey() {

Review comment:
       Should this method return a `SecretKey` instead of forcing the caller to perform the conversion, or is the goal to keep this more generic?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyHolder.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.properties;
+
+import java.util.function.Supplier;
+
+/**
+ * A base class that encapsulates some level of sensitive properties.

Review comment:
       A little more clarity would be helpful, perhaps renaming this class to something like `SensitivePropertyAware` might helpful.  Perhaps updating the comment to say something like: "encapsulates access to common Sensitive Property methods".

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.properties;
+
+import java.nio.file.Path;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+public class BootstrapProperties {

Review comment:
       Depending on the interface definition, `BootstrapProperties` could implement `ApplicationProperties`, although that might be confusing when it comes to semantics.  From a functional standpoint however, it seems like there could be a shared interface.  A class comment might be helpful.

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedApplicationProperties.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.properties;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Represents a protected set of ApplicationProperties, with methods regarding which sensitive properties
+ * are protected.
+ * @param <T> The ApplicationProperties type
+ */
+public interface ProtectedApplicationProperties<T extends ApplicationProperties> {

Review comment:
       Understanding the challenge of naming each of these components, `ProtectedApplicationProperties` appears to suggest an extension of `ApplicationProperties`.  What about naming this `ProtectedProperties`?

##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.properties;
+
+import java.nio.file.Path;
+import java.util.Enumeration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+public class BootstrapProperties {
+    private static final String PROPERTY_KEY_FORMAT = "%s.%s";
+    private static final String BOOTSTRAP_SENSITIVE_KEY = "bootstrap.sensitive.key";
+
+    private final String propertyPrefix;
+    private final Properties properties;
+    private final Path configFilePath;
+
+    public BootstrapProperties(final String propertyPrefix, final Properties properties, final Path configFilePath) {
+        Objects.requireNonNull(propertyPrefix, "Property prefix is required");
+        Objects.requireNonNull(properties, "Properties are required");

Review comment:
       The `Objects.requireNonNull()` method returns the value, so these lines could be combined with the variable assignment.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProtectionScheme.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.properties;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+/**
+ * A scheme for protecting sensitive properties.  Each scheme is intended to be backed by an implementation of
+ * SensitivePropertyProvider.
+ */
+public enum SensitivePropertyProtectionScheme {

Review comment:
       For brevity, perhaps just `PropertyProtectionScheme` is sufficient?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProtectionScheme.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.properties;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+/**
+ * A scheme for protecting sensitive properties.  Each scheme is intended to be backed by an implementation of
+ * SensitivePropertyProvider.
+ */
+public enum SensitivePropertyProtectionScheme {
+    AES_GCM("aes/gcm/[0-9]+", "aes/gcm/%s", "AES Sensitive Property Provider", true);

Review comment:
       For the identifier pattern, the number should be more narrowly defined, either `[0-9]{3}` to require three numbers, or perhaps more limited to allow only 128, 192, or 256.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactory.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.properties;
+
+import java.util.Collection;
+
+public interface SensitivePropertyProviderFactory {
+
+    /**
+     * Gives the appropriate SensitivePropertyProvider, given a protection scheme.
+     * @param protectionScheme The protection scheme to use
+     * @return The appropriate SensitivePropertyProvider
+     */
+    SensitivePropertyProvider getProvider(SensitivePropertyProtectionScheme protectionScheme);
+
+    /**
+     * Gives the appropriate SensitivePropertyProvider, given a protection scheme identifier.
+     * @param protectionSchemeIdentifier The identifier of the protection scheme to use
+     * @return The appropriate SensitivePropertyProvider
+     */
+    SensitivePropertyProvider getProvider(String protectionSchemeIdentifier);

Review comment:
       Is it necessary to support this method as well as the one above which requires the more concrete protection scheme parameter?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.properties;
+
+import org.apache.nifi.security.kms.CryptoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+public class StandardSensitivePropertyProviderFactory implements SensitivePropertyProviderFactory {
+    private static final Logger logger = LoggerFactory.getLogger(StandardSensitivePropertyProviderFactory.class);
+
+    private final Optional<String> keyHex;
+    private final Supplier<BootstrapProperties> bootstrapPropertiesSupplier;
+    private final Map<SensitivePropertyProtectionScheme, SensitivePropertyProvider> providerMap;
+
+    /**
+     * Creates a StandardSensitivePropertyProviderFactory using the default bootstrap.conf location and
+     * the keyHex extracted from this bootstrap.conf.
+     */
+    public static SensitivePropertyProviderFactory withDefaults() {
+        return withKeyAndBootstrapSupplier(null, null);
+    }
+
+    /**
+     * Creates a StandardSensitivePropertyProviderFactory using only the provided secret key hex.  The default
+     * bootstrap.conf will be used for any providers that may require it, but the provided keyHex will be used instead
+     * of the one from the default bootstrap.conf.
+     * @param keyHex The secret key hex for encrypting properties
+     * @return A StandardSensitivePropertyProviderFactory
+     */
+    public static SensitivePropertyProviderFactory withKey(final String keyHex) {
+        return new StandardSensitivePropertyProviderFactory(keyHex, null);
+    }
+
+    /**
+     * Creates a new StandardSensitivePropertyProviderFactory using a separate keyHex and provided bootstrap.conf.
+     * The provided keyHex will be used instead of the one from the bootstrap.conf.
+     * @param keyHex The secret key hex for encrypting properties
+     * @param bootstrapPropertiesSupplier A supplier for the BootstrapProperties that represent bootstrap.conf.
+     *                                    If the supplier returns null, the default bootstrap.conf will be used instead.
+     * @return A StandardSensitivePropertyProviderFactory
+     */
+    public static SensitivePropertyProviderFactory withKeyAndBootstrapSupplier(final String keyHex,
+                                                                               final Supplier<BootstrapProperties> bootstrapPropertiesSupplier) {
+        return new StandardSensitivePropertyProviderFactory(keyHex, bootstrapPropertiesSupplier);
+    }
+
+    private StandardSensitivePropertyProviderFactory(final String keyHex, final Supplier<BootstrapProperties> bootstrapPropertiesSupplier) {
+        this.keyHex = Optional.ofNullable(keyHex);
+        this.bootstrapPropertiesSupplier = bootstrapPropertiesSupplier == null ? () -> null : bootstrapPropertiesSupplier;
+        this.providerMap = new HashMap<>();
+    }
+
+    private String getKeyHex() {
+        return keyHex.orElseGet(() -> getBootstrapProperties().getBootstrapSensitiveKey()
+                .orElseThrow(() -> new SensitivePropertyProtectionException("Could not read root key from bootstrap.conf")));
+    }
+
+    /**
+     * Returns the configured bootstrap properties, or the default bootstrap.conf properties if
+     * not provided.
+     * @return The bootstrap.conf properties
+     */
+    private BootstrapProperties getBootstrapProperties() {
+        return Optional.ofNullable(bootstrapPropertiesSupplier.get()).orElseGet(() -> {
+            try {
+                return CryptoUtils.loadBootstrapProperties("");

Review comment:
       Although it involves more changes, this seems like the time to refactor bootstrap properties loading out of the `CryptoUtils` class.  Having an empty string as a parameter is also confusing.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
##########
@@ -16,13 +16,14 @@
  */
 package org.apache.nifi.cluster.protocol.spring;
 
-import java.util.concurrent.TimeUnit;
 import org.apache.nifi.io.socket.ServerSocketConfiguration;
 import org.apache.nifi.security.util.StandardTlsConfiguration;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.springframework.beans.factory.FactoryBean;
 
+import java.util.concurrent.TimeUnit;

Review comment:
       It looks like this change could be reverted since it is just reordering imports.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProtector.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.properties;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Encapsulates methods needed to protect application properties.
+ * @param <T> The ProtectedApplicationProperties type
+ * @param <U> The ApplicationProperties type
+ */
+public interface SensitivePropertyProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties> {
+
+    /**
+     * Returns the number of properties, excluding protection scheme properties.
+     * <p>
+     * Example:
+     * <p>
+     * key: E(value, key)
+     * key.protected: aes/gcm/256
+     * key2: value2
+     * <p>
+     * would return size 2
+     *
+     * @return the count of real properties
+     */
+    int size();
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the complete set of property keys, including any protection keys (i.e. 'x.y.z.protected').
+     *
+     * @return the set of property keys
+     */
+    Set<String> getPropertyKeysIncludingProtectionSchemes();
+
+    /**
+     * Returns a list of the keys identifying "sensitive" properties. There is a default list,
+     * and additional keys can be provided in the {@code nifi.sensitive.props.additional.keys} property in the ApplicationProperties.
+     *
+     * @return the list of sensitive property keys
+     */
+    List<String> getSensitivePropertyKeys();
+
+    /**
+     * Returns a list of the keys identifying "sensitive" properties. There is a default list,
+     * and additional keys can be provided in the {@code nifi.sensitive.props.additional.keys} property in the ApplicationProperties.
+     *
+     * @return the list of sensitive property keys
+     */
+    List<String> getPopulatedSensitivePropertyKeys();
+
+    /**
+     * Returns true if any sensitive keys are protected.
+     *
+     * @return true if any key is protected; false otherwise
+     */
+    boolean hasProtectedKeys();
+
+    /**
+     * Returns the unique set of all protection schemes currently in use for this instance.
+     *
+     * @return the set of protection schemes
+     */
+    Set<String> getProtectionSchemes();
+
+    /**
+     * Returns a Map of the keys identifying "sensitive" properties that are currently protected and the "protection" key for each.
+     * This may or may not include all properties marked as sensitive.
+     *
+     * @return the Map of protected property keys and the protection identifier for each
+     */
+    Map<String, String> getProtectedPropertyKeys();
+
+    /**
+     * Returns the local provider cache (null-safe) as a Map of protection schemes -> implementations.
+     *
+     * @return the map
+     */
+    Map<String, SensitivePropertyProvider> getSensitivePropertyProviders();
+
+    /**
+     * Returns a percentage of the total number of populated properties marked as sensitive that are currently protected.
+     *
+     * @return the percent of sensitive properties marked as protected
+     */
+    int getPercentOfSensitivePropertiesProtected();

Review comment:
       Although this is used for informational purposes, is it necessary at the interface level?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
##########
@@ -87,11 +84,18 @@ private static JAXBContext initializeJaxbContext() {
         }
     }
 
-    private NiFiProperties properties;
     private ExtensionManager extensionManager;
     private LoginIdentityProvider loginIdentityProvider;
     private final Map<String, LoginIdentityProvider> loginIdentityProviders = new HashMap<>();
 
+    public NiFiProperties getProperties() {

Review comment:
       Is this get method necessary?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactoryTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.properties;
+
+import org.apache.nifi.util.NiFiProperties;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.io.IOUtil;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.Security;
+import java.util.Properties;
+import java.util.function.Supplier;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class StandardSensitivePropertyProviderFactoryTest {
+
+    private SensitivePropertyProviderFactory factory;
+
+    private static final String BOOTSTRAP_KEY_HEX = "0123456789ABCDEFFEDCBA9876543210";
+    private static final String AD_HOC_KEY_HEX = "123456789ABCDEFFEDCBA98765432101";
+
+    private static Path tempConfDir;
+    private static Path mockBootstrapConf;
+    private static Path mockNifiProperties;
+
+    private static NiFiProperties niFiProperties;
+
+    @BeforeClass
+    public static void initOnce() throws IOException {
+        Security.addProvider(new BouncyCastleProvider());
+        tempConfDir = Files.createTempDirectory("conf");
+        mockBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
+
+        mockNifiProperties = Files.createTempFile("nifi", ".properties").toAbsolutePath();
+
+        mockBootstrapConf = Files.move(mockBootstrapConf, tempConfDir.resolve("bootstrap.conf"));
+        mockNifiProperties = Files.move(mockNifiProperties, tempConfDir.resolve("nifi.properties"));
+
+        IOUtil.writeText("nifi.bootstrap.sensitive.key=" + BOOTSTRAP_KEY_HEX, mockBootstrapConf.toFile());
+        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, mockNifiProperties.toString());
+
+        niFiProperties = new NiFiProperties();
+    }
+
+    @AfterClass
+    public static void tearDownOnce() throws IOException {
+        Files.deleteIfExists(mockBootstrapConf);
+        Files.deleteIfExists(mockNifiProperties);
+        Files.deleteIfExists(tempConfDir);
+        System.clearProperty(NiFiProperties.PROPERTIES_FILE_PATH);
+    }
+
+    /**
+     * Configures the factory using the default bootstrap location.
+     */
+    private void configureDefaultFactory() {
+        factory = StandardSensitivePropertyProviderFactory.withDefaults();
+    }
+
+    /**
+     * Configures the factory using an ad hoc key hex.
+     */
+    private void configureAdHocKeyFactory() {
+        factory = StandardSensitivePropertyProviderFactory.withKey(AD_HOC_KEY_HEX);
+    }
+
+    /**
+     * Configures the factory using an ad hoc key hex and bootstrap.conf properties.  The key should override
+     * the on in the bootstrap.conf.
+     */
+    private void configureAdHocKeyAndPropertiesFactory() throws IOException {
+        factory = StandardSensitivePropertyProviderFactory.withKeyAndBootstrapSupplier(AD_HOC_KEY_HEX, mockBootstrapProperties());
+    }
+
+    private Supplier<BootstrapProperties> mockBootstrapProperties() throws IOException {
+        final Properties bootstrapProperties = new Properties();
+        try (final InputStream inputStream = Files.newInputStream(mockBootstrapConf)) {
+            bootstrapProperties.load(inputStream);
+            return () -> new BootstrapProperties("nifi", bootstrapProperties, mockBootstrapConf);
+        }
+    }
+
+    @Test
+    public void testAES_GCM() throws IOException {
+        configureDefaultFactory();
+
+        final SensitivePropertyProvider spp = factory.getProvider(SensitivePropertyProtectionScheme.AES_GCM);
+        assertNotNull(spp);
+        assertTrue(spp.isSupported());
+
+        final String cleartext = "test";
+        assertEquals(cleartext, spp.unprotect(spp.protect(cleartext)));
+        assertNotEquals(cleartext, spp.protect(cleartext));
+        assertEquals("aes/gcm/128", spp.getIdentifierKey());

Review comment:
       Recommend declaring a method variable or static variable for `aes/gcm/128` and reusing in this method.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Class performing unprotection activities before returning a clean
+ * implementation of {@link ApplicationProperties}.
+ * This encapsulates the sensitive property access logic from external consumers
+ * of {@code ApplicationProperties}.
+ *
+ * @param <T> The type of protected application properties
+ * @param <U> The type of standard application properties that backs the protected application properties
+ */
+public class ApplicationPropertiesProtector<T extends ProtectedApplicationProperties<U>, U extends ApplicationProperties>
+        implements SensitivePropertyProtector<T, U> {
+    private static final Logger logger = LoggerFactory.getLogger(ApplicationPropertiesProtector.class);
+
+    private T protectedApplicationProperties;
+
+    private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+    /**
+     * Creates an instance containing the provided {@link ProtectedApplicationProperties}.
+     *
+     * @param protectedApplicationProperties the ProtectedApplicationProperties to contain
+     */
+    public ApplicationPropertiesProtector(final T protectedApplicationProperties) {
+        this.protectedApplicationProperties = protectedApplicationProperties;
+        logger.debug("Loaded {} properties (including {} protection schemes) into {}", getPropertyKeysIncludingProtectionSchemes().size(),
+                getProtectedPropertyKeys().size(), this.getClass().getName());
+    }
+
+    /**
+     * Returns the sibling property key which specifies the protection scheme for this key.
+     * <p>
+     * Example:
+     * <p>
+     * nifi.sensitive.key=ABCXYZ
+     * nifi.sensitive.key.protected=aes/gcm/256
+     * <p>
+     * nifi.sensitive.key -> nifi.sensitive.key.protected
+     *
+     * @param key the key identifying the sensitive property
+     * @return the key identifying the protection scheme for the sensitive property
+     */
+    public static String getProtectionKey(final String key) {
+        if (key == null || key.isEmpty()) {
+            throw new IllegalArgumentException("Cannot find protection key for null key");
+        }
+
+        return key + ".protected";
+    }
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    @Override
+    public Set<String> getPropertyKeys() {
+        Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+        filteredKeys.removeIf(p -> p.endsWith(".protected"));
+        return filteredKeys;
+    }
+
+    @Override
+    public int size() {
+        return getPropertyKeys().size();
+    }
+
+    @Override
+    public Set<String> getPropertyKeysIncludingProtectionSchemes() {
+        return protectedApplicationProperties.getUnderlyingProperties().getPropertyKeys();
+    }
+
+    /**
+     * Splits a single string containing multiple property keys into a List. Delimited by ',' or ';' and ignores leading and trailing whitespace around delimiter.
+     *
+     * @param multipleProperties a single String containing multiple properties, i.e. "nifi.property.1; nifi.property.2, nifi.property.3"
+     * @return a List containing the split and trimmed properties
+     */
+    private static List<String> splitMultipleProperties(final String multipleProperties) {
+        if (multipleProperties == null || multipleProperties.trim().isEmpty()) {
+            return new ArrayList<>(0);
+        } else {
+            List<String> properties = new ArrayList<>(asList(multipleProperties.split("\\s*[,;]\\s*")));
+            for (int i = 0; i < properties.size(); i++) {
+                properties.set(i, properties.get(i).trim());
+            }
+            return properties;
+        }
+    }
+
+    private String getProperty(final String key) {
+        return protectedApplicationProperties.getUnderlyingProperties().getProperty(key);
+    }
+
+    private String getAdditionalSensitivePropertiesKeys() {
+        return getProperty(protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName());
+    }
+
+    private String getAdditionalSensitivePropertiesKeysName() {
+        return protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+    }
+
+    @Override
+    public List<String> getSensitivePropertyKeys() {
+        final String additionalPropertiesString = getAdditionalSensitivePropertiesKeys();
+        final String additionalPropertiesKeyName = protectedApplicationProperties.getAdditionalSensitivePropertiesKeysName();
+        if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+            return protectedApplicationProperties.getDefaultSensitiveProperties();
+        } else {
+            List<String> additionalProperties = splitMultipleProperties(additionalPropertiesString);
+            /* Remove this key if it was accidentally provided as a sensitive key
+             * because we cannot protect it and read from it
+            */
+            if (additionalProperties.contains(additionalPropertiesKeyName)) {
+                logger.warn("The key '{}' contains itself. This is poor practice and should be removed", additionalPropertiesKeyName);
+                additionalProperties.remove(additionalPropertiesKeyName);
+            }
+            additionalProperties.addAll(protectedApplicationProperties.getDefaultSensitiveProperties());
+            return additionalProperties;
+        }
+    }
+
+    @Override
+    public List<String> getPopulatedSensitivePropertyKeys() {
+        List<String> allSensitiveKeys = getSensitivePropertyKeys();
+        return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasProtectedKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+        for (String k : sensitiveKeys) {
+            if (isPropertyProtected(k)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getProtectedPropertyKeys() {
+        final List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+        // This is the Java 8 way, but can likely be optimized (and not sure of correctness)
+        // Map<String, String> protectedProperties = sensitiveKeys.stream().filter(key ->
+        // getProperty(getProtectionKey(key)) != null).collect(Collectors.toMap(Function.identity(), key ->
+        // getProperty(getProtectionKey(key))));
+
+        // Groovy
+        // Map<String, String> groovyProtectedProperties = sensitiveKeys.collectEntries { key ->
+        // [(key): getProperty(getProtectionKey(key))] }.findAll { k, v -> v }
+
+        // Traditional way
+        final Map<String, String> traditionalProtectedProperties = new HashMap<>();
+        for (final String key : sensitiveKeys) {
+            final String protection = getProperty(getProtectionKey(key));
+            if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+                traditionalProtectedProperties.put(key, protection);
+            }
+        }
+
+        return traditionalProtectedProperties;
+    }
+
+    @Override
+    public Set<String> getProtectionSchemes() {
+        return new HashSet<>(getProtectedPropertyKeys().values());
+    }
+
+    @Override
+    public int getPercentOfSensitivePropertiesProtected() {
+        return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+    }
+
+    @Override
+    public boolean isPropertySensitive(final String key) {
+        // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this will loop infinitely
+        return key != null && !key.equals(getAdditionalSensitivePropertiesKeysName()) && getSensitivePropertyKeys().contains(key.trim());
+    }
+
+    /**
+     * Returns true if the property identified by this key is considered protected in this instance of {@code NiFiProperties}.
+     * The property value is protected if the key is sensitive and the sibling key of key.protected is present.
+     *
+     * @param key the key
+     * @return true if it is currently marked as protected
+     * @see ApplicationPropertiesProtector#getSensitivePropertyKeys()
+     */
+    @Override
+    public boolean isPropertyProtected(final String key) {
+        return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+    }
+
+    @Override
+    public U getUnprotectedProperties() throws SensitivePropertyProtectionException {
+        if (hasProtectedKeys()) {
+            logger.info("There are {} protected properties of {} sensitive properties ({}%)",
+                    getProtectedPropertyKeys().size(),
+                    getSensitivePropertyKeys().size(),
+                    getPercentOfSensitivePropertiesProtected());
+
+            final Properties rawProperties = new Properties();
+
+            final Set<String> failedKeys = new HashSet<>();
+
+            for (final String key : getPropertyKeys()) {
+                /* Three kinds of keys
+                 * 1. protection schemes -- skip
+                 * 2. protected keys -- unprotect and copy
+                 * 3. normal keys -- copy over
+                 */
+                if (key.endsWith(".protected")) {
+                    // Do nothing
+                } else if (isPropertyProtected(key)) {
+                    try {
+                        rawProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+                    } catch (final SensitivePropertyProtectionException e) {
+                        logger.warn("Failed to unprotect '{}'", key, e);
+                        failedKeys.add(key);
+                    }
+                } else {
+                    rawProperties.setProperty(key, getProperty(key));
+                }
+            }
+
+            if (!failedKeys.isEmpty()) {
+                if (failedKeys.size() > 1) {
+                    logger.warn("Combining {} failed keys [{}] into single exception", failedKeys.size(), StringUtils.join(failedKeys, ", "));
+                    throw new MultipleSensitivePropertyProtectionException("Failed to unprotect keys", failedKeys);
+                } else {
+                    throw new SensitivePropertyProtectionException("Failed to unprotect key " + failedKeys.iterator().next());
+                }
+            }
+
+            final U unprotected = protectedApplicationProperties.createApplicationProperties(rawProperties);
+
+            return unprotected;
+        } else {
+            logger.debug("No protected properties");
+            return protectedApplicationProperties.getUnderlyingProperties();
+        }
+    }
+
+    @Override
+    public void addSensitivePropertyProvider(final SensitivePropertyProvider sensitivePropertyProvider) {
+        if (sensitivePropertyProvider == null) {
+            throw new IllegalArgumentException("Cannot add null SensitivePropertyProvider");
+        }
+
+        if (getSensitivePropertyProviders().containsKey(sensitivePropertyProvider.getIdentifierKey())) {
+            throw new UnsupportedOperationException("Cannot overwrite existing sensitive property provider registered for " + sensitivePropertyProvider.getIdentifierKey());
+        }
+
+        getSensitivePropertyProviders().put(sensitivePropertyProvider.getIdentifierKey(), sensitivePropertyProvider);
+    }
+
+    @Override
+    public String toString() {
+        final Set<String> providers = getSensitivePropertyProviders().keySet();
+        return new StringBuilder("ApplicationPropertiesProtector instance with ")
+                .append(size()).append(" properties (")
+                .append(getProtectedPropertyKeys().size())
+                .append(" protected) and ")
+                .append(providers.size())
+                .append(" sensitive property providers: ")
+                .append(StringUtils.join(providers, ", "))
+                .toString();
+    }
+
+    @Override
+    public Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+        if (localProviderCache == null) {
+            localProviderCache = new HashMap<>();
+        }
+
+        return localProviderCache;
+    }
+
+    private SensitivePropertyProvider getSensitivePropertyProvider(final String protectionScheme) {
+        if (isProviderAvailable(protectionScheme)) {
+            return getSensitivePropertyProviders().get(protectionScheme);
+        } else {
+            throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+        }
+    }
+
+    private boolean isProviderAvailable(final String protectionScheme) {
+        return getSensitivePropertyProviders().containsKey(protectionScheme);
+    }
+
+    /**
+     * If the value is protected, unprotects it and returns it. If not, returns the original value.
+     *
+     * @param key            the retrieved property key
+     * @param retrievedValue the retrieved property value
+     * @return the unprotected value
+     */
+    private String unprotectValue(final String key, final String retrievedValue) {
+        // Checks if the key is sensitive and marked as protected
+        if (isPropertyProtected(key)) {
+            final String protectionScheme = getProperty(getProtectionKey(key));
+
+            // No provider registered for this scheme, so just return the value
+            if (!isProviderAvailable(protectionScheme)) {
+                logger.warn("No provider available for {} so passing the protected {} value back", protectionScheme, key);

Review comment:
       Should this be a warning as opposed to throwing some subclass of RuntimeException?  Perhaps this validation could be performed earlier, but it seems like a configuration error that should cause an exception.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
##########
@@ -87,12 +84,19 @@ private static JAXBContext initializeJaxbContext() {
     }
 
     private Authorizer authorizer;
-    private NiFiProperties properties;
     private ExtensionManager extensionManager;
     private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
     private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
     private final Map<String, Authorizer> authorizers = new HashMap<>();
 
+    public NiFiProperties getProperties() {
+        return properties;
+    }

Review comment:
       Is this get method necessary?

##########
File path: nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
##########
@@ -30,8 +30,8 @@
             <version>1.14.0-SNAPSHOT</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>

Review comment:
       Is it intentional to replace `nifi-properties` here with `nifi-registry-properties`?  Should both dependencies be declared?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648525049



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedApplicationProperties.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.properties;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Represents a protected set of ApplicationProperties, with methods regarding which sensitive properties
+ * are protected.
+ * @param <T> The ApplicationProperties type
+ */
+public interface ProtectedApplicationProperties<T extends ApplicationProperties> {

Review comment:
       Yes, I believe I started out with one extending the other, but now that I've disconnected them, I agree with ProtectedProperties.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648395519



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationPropertiesBase.java
##########
@@ -18,64 +18,52 @@
 
 import java.util.Enumeration;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import org.apache.nifi.util.NiFiProperties;
 
-public class StandardNiFiProperties extends NiFiProperties {
+public abstract class ApplicationPropertiesBase implements ApplicationProperties {

Review comment:
       I'm liking StandardApplicationProperties now that it doesn't have to be abstract.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648571830



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
##########
@@ -87,12 +84,19 @@ private static JAXBContext initializeJaxbContext() {
     }
 
     private Authorizer authorizer;
-    private NiFiProperties properties;
     private ExtensionManager extensionManager;
     private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
     private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
     private final Map<String, Authorizer> authorizers = new HashMap<>();
 
+    public NiFiProperties getProperties() {
+        return properties;
+    }

Review comment:
       I believe Spring works with just the set method defined currently, so if this is not necessary, it could be removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#issuecomment-859899572






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5131: NIFI-8651: Refactor Sensitive Properties Providers for extension

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5131:
URL: https://github.com/apache/nifi/pull/5131#discussion_r648376393



##########
File path: nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ApplicationProperties.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.properties;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * A base interface for configuration properties of an application (e.g. NiFi or NiFi Registry).
+ */
+public interface ApplicationProperties {
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key);
+
+    /**
+     * Retrieves the property value for the given property key.
+     *
+     * @param key the key of property value to lookup
+     * @param defaultValue The default value to use if the property does not exist
+     * @return value of property at given key or null if not found
+     */
+    String getProperty(String key, String defaultValue);
+
+    /**
+     * Retrieves all known property keys.
+     *
+     * @return all known property keys
+     */
+    Set<String> getPropertyKeys();
+
+    /**
+     * Returns the number of properties.
+     * @return The number of properties
+     */
+    int size();

Review comment:
       It appears it's only used in NiFiProperties itself, so it looks like no.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org