NIFIREG-61 Add support for encrypted config files
Allows sensitive property values to be encrypted in the following
configuration files:
- nifi-registry.properties
- identity-providers.xml
- authorizers.xml
A master decryption key can be configured that allows
decrypting protected properties at runtime, specifically:
- Adds CryptoKeyProvider interface for injecting key into application
- Provides implementation that is backed by bootstrap.conf
- Provides implementation that keeps key in memory
- Provides mechanism for removing CryptoKeyProvider from scope
after Application Context is done loading
NIFIREG-61 Simplify Master Key Loading
Simplifies the code associated with loading the master crypto key
to standardize on using the bootstrap.conf file.
This closes #51.
Signed-off-by: Bryan Bende <bbende@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/nifi-registry/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi-registry/commit/64211451
Tree: http://git-wip-us.apache.org/repos/asf/nifi-registry/tree/64211451
Diff: http://git-wip-us.apache.org/repos/asf/nifi-registry/diff/64211451
Branch: refs/heads/master
Commit: 64211451cc87ccbd4f0a3762b2ffca17781cddd2
Parents: a3f01fa
Author: Kevin Doran <kdoran.apache@gmail.com>
Authored: Tue Dec 5 14:44:24 2017 -0500
Committer: Bryan Bende <bbende@apache.org>
Committed: Tue Dec 26 14:57:09 2017 -0500
----------------------------------------------------------------------
.../registry/bootstrap/RunNiFiRegistry.java | 47 +-
.../authentication/IdentityProviderFactory.java | 38 +-
.../authorization/AuthorizerFactory.java | 36 +-
.../SensitivePropertyProviderConfiguration.java | 67 ++
.../src/main/xsd/authorizers.xsd | 1 +
.../apache/nifi/registry/jetty/JettyServer.java | 8 +-
nifi-registry-properties/pom.xml | 42 ++
.../AESSensitivePropertyProvider.java | 265 +++++++
.../AESSensitivePropertyProviderFactory.java | 54 ++
...pleSensitivePropertyProtectionException.java | 129 ++++
.../NiFiRegistryPropertiesLoader.java | 148 ++++
.../ProtectedNiFiRegistryProperties.java | 528 +++++++++++++
.../SensitivePropertyProtectionException.java | 89 +++
.../properties/SensitivePropertyProvider.java | 52 ++
.../SensitivePropertyProviderFactory.java | 23 +
.../crypto/BootstrapFileCryptoKeyProvider.java | 81 ++
.../security/crypto/CryptoKeyLoader.java | 87 +++
.../security/crypto/CryptoKeyProvider.java | 68 ++
.../crypto/MissingCryptoKeyException.java | 47 ++
...SSensitivePropertyProviderFactoryTest.groovy | 81 ++
.../AESSensitivePropertyProviderTest.groovy | 471 ++++++++++++
.../NiFiRegistryPropertiesGroovyTest.groovy | 121 +++
...iFiRegistryPropertiesLoaderGroovyTest.groovy | 264 +++++++
.../ProtectedNiFiPropertiesGroovyTest.groovy | 739 +++++++++++++++++++
.../crypto/CryptoKeyLoaderGroovyTest.groovy | 121 +++
.../src/test/resources/conf/bootstrap.conf | 60 ++
.../bootstrap.unreadable_file_permissions.conf | 22 +
.../conf/bootstrap.with_missing_key.conf | 60 ++
.../conf/bootstrap.with_missing_key_line.conf | 60 ++
.../resources/conf/nifi-registry.properties | 45 ++
...ry.with_additional_sensitive_keys.properties | 55 ++
...ive_props_fully_protected_aes_128.properties | 43 ++
...sensitive_props_protected_aes_128.properties | 43 ++
..._props_protected_aes_128_password.properties | 43 ++
...sensitive_props_protected_aes_256.properties | 43 ++
..._protected_aes_multiple_malformed.properties | 43 ++
...ps_protected_aes_single_malformed.properties | 43 ++
...sensitive_props_protected_unknown.properties | 43 ++
....with_sensitive_props_unprotected.properties | 41 +
...tive_props_unprotected_extra_line.properties | 42 ++
.../src/main/resources/conf/bootstrap.conf | 3 +
.../resources/conf/nifi-registry.properties | 3 +
.../org/apache/nifi/registry/NiFiRegistry.java | 142 +---
nifi-registry-web-api/pom.xml | 4 +-
.../registry/NiFiRegistryApiApplication.java | 1 +
.../NiFiRegistryMasterKeyProviderFactory.java | 67 ++
.../nifi/registry/web/api/SecureLdapIT.java | 18 +-
.../conf/secure-ldap/authorizers.protected.xml | 221 ++++++
.../resources/conf/secure-ldap/bootstrap.conf | 60 ++
.../identity-providers.protected.xml | 89 +++
.../conf/secure-ldap/nifi-registry.properties | 4 +-
51 files changed, 4748 insertions(+), 157 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java b/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java
index c7bec04..c6d92ea 100644
--- a/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java
+++ b/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java
@@ -16,8 +16,13 @@
*/
package org.apache.nifi.registry.bootstrap;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.bootstrap.util.OSUtils;
+import org.apache.nifi.registry.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import java.io.BufferedReader;
-import java.io.BufferedWriter;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
@@ -34,12 +39,7 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.nio.file.Path;
-import java.nio.file.attribute.FileAttribute;
import java.nio.file.attribute.PosixFilePermission;
-import java.nio.file.attribute.PosixFilePermissions;
-import java.nio.file.FileAlreadyExistsException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
@@ -58,11 +58,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.bootstrap.util.OSUtils;
-import org.apache.nifi.registry.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* <p>
@@ -951,36 +946,6 @@ public class RunNiFiRegistry {
cmd.add("-Dapp=NiFiRegistry");
cmd.add("-Dorg.apache.nifi.registry.bootstrap.config.log.dir=" + nifiRegistryLogDir);
cmd.add("org.apache.nifi.registry.NiFiRegistry");
- if (props.containsKey(NIFI_REGISTRY_BOOTSTRAP_SENSITIVE_KEY) && !StringUtils.isBlank(props.get(NIFI_REGISTRY_BOOTSTRAP_SENSITIVE_KEY))) {
- Path sensitiveKeyFile = Paths.get(confDir+"/sensitive.key");
-
- try {
- // Initially create file with the empty permission set (so nobody can get a file descriptor on it):
- Set<PosixFilePermission> perms = new HashSet<PosixFilePermission>();
- FileAttribute<Set<PosixFilePermission>> attr = PosixFilePermissions.asFileAttribute(perms);
- sensitiveKeyFile = Files.createFile(sensitiveKeyFile, attr);
-
- // Then, once created, add owner-only rights:
- perms.add(PosixFilePermission.OWNER_WRITE);
- perms.add(PosixFilePermission.OWNER_READ);
- attr = PosixFilePermissions.asFileAttribute(perms);
- Files.setPosixFilePermissions(sensitiveKeyFile, perms);
-
- } catch (final FileAlreadyExistsException faee) {
- cmdLogger.error("The sensitive.key file {} already exists. That shouldn't have been. Aborting.", sensitiveKeyFile);
- System.exit(1);
- } catch (final Exception e) {
- cmdLogger.error("Other failure relating to setting permissions on {}. "
- + "(so that only the owner can read it). "
- + "This is fatal to the bootstrap process for security reasons. Exception was: {}", sensitiveKeyFile, e);
- System.exit(1);
- }
-
- BufferedWriter sensitiveKeyWriter = Files.newBufferedWriter(sensitiveKeyFile, StandardCharsets.UTF_8);
- sensitiveKeyWriter.write(props.get(NIFI_REGISTRY_BOOTSTRAP_SENSITIVE_KEY));
- sensitiveKeyWriter.close();
- cmd.add("-K " + sensitiveKeyFile.toFile().getAbsolutePath());
- }
builder.command(cmd);
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
index 720bd9c..3c2a3f4 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
@@ -19,6 +19,8 @@ package org.apache.nifi.registry.security.authentication;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.registry.extension.ExtensionManager;
import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.registry.properties.SensitivePropertyProvider;
import org.apache.nifi.registry.security.authentication.annotation.IdentityProviderContext;
import org.apache.nifi.registry.security.authentication.generated.IdentityProviders;
import org.apache.nifi.registry.security.authentication.generated.Property;
@@ -31,6 +33,7 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
import org.springframework.context.annotation.Primary;
+import org.springframework.lang.Nullable;
import org.xml.sax.SAXException;
import javax.xml.XMLConstants;
@@ -68,13 +71,18 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
private NiFiRegistryProperties properties;
private ExtensionManager extensionManager;
+ private SensitivePropertyProvider sensitivePropertyProvider;
private IdentityProvider identityProvider;
private final Map<String, IdentityProvider> identityProviders = new HashMap<>();
@Autowired
- public IdentityProviderFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
+ public IdentityProviderFactory(
+ final NiFiRegistryProperties properties,
+ final ExtensionManager extensionManager,
+ @Nullable final SensitivePropertyProvider sensitivePropertyProvider) {
this.properties = properties;
this.extensionManager = extensionManager;
+ this.sensitivePropertyProvider = sensitivePropertyProvider;
if (this.properties == null) {
throw new IllegalStateException("NiFiRegistryProperties cannot be null");
@@ -90,9 +98,8 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
return identityProviders.get(identifier);
}
- @Primary
@Bean
-// @Bean("LoginIdentityProvider")
+ @Primary
public IdentityProvider getIdentityProvider() throws Exception {
if (identityProvider == null) {
// look up the login identity provider to use
@@ -186,7 +193,12 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
final Map<String, String> providerProperties = new HashMap<>();
for (final Property property : provider.getProperty()) {
- providerProperties.put(property.getName(), property.getValue());
+ if (!StringUtils.isBlank(property.getEncryption())) {
+ String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+ providerProperties.put(property.getName(), decryptedValue);
+ } else {
+ providerProperties.put(property.getName(), property.getValue());
+ }
}
return new StandardIdentityProviderConfigurationContext(provider.getIdentifier(), this, providerProperties);
@@ -258,4 +270,22 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
}
}
+ private String decryptValue(String cipherText, String encryptionScheme) throws SensitivePropertyProtectionException {
+ if (sensitivePropertyProvider == null) {
+ throw new SensitivePropertyProtectionException("Sensitive Property Provider dependency was never wired, so protected " +
+ "properties cannot be decrypted. This usually indicates that a master key for this NiFi Registry was not " +
+ "detected and configured during the bootstrap startup sequence. Contact the system administrator.");
+ }
+
+ if (!sensitivePropertyProvider.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
+ throw new SensitivePropertyProtectionException("Identity Provider configuration XML was protected using " +
+ encryptionScheme +
+ ", but the configured Sensitive Property Provider supports " +
+ sensitivePropertyProvider.getIdentifierKey() +
+ ". Cannot configure this Identity Provider due to failing to decrypt protected configuration properties.");
+ }
+
+ return sensitivePropertyProvider.unprotect(cipherText);
+ }
+
}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
index 49b5688..ff412cb 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
@@ -19,6 +19,8 @@ package org.apache.nifi.registry.security.authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.registry.extension.ExtensionManager;
import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.registry.properties.SensitivePropertyProvider;
import org.apache.nifi.registry.provider.StandardProviderFactory;
import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
@@ -34,6 +36,7 @@ import org.springframework.beans.factory.DisposableBean;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
+import org.springframework.lang.Nullable;
import org.xml.sax.SAXException;
import javax.xml.XMLConstants;
@@ -80,6 +83,7 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
private final NiFiRegistryProperties properties;
private final ExtensionManager extensionManager;
+ private final SensitivePropertyProvider sensitivePropertyProvider;
private Authorizer authorizer;
private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
@@ -87,9 +91,14 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
private final Map<String, Authorizer> authorizers = new HashMap<>();
@Autowired
- public AuthorizerFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
+ public AuthorizerFactory(
+ final NiFiRegistryProperties properties,
+ final ExtensionManager extensionManager,
+ @Nullable final SensitivePropertyProvider sensitivePropertyProvider) {
+
this.properties = properties;
this.extensionManager = extensionManager;
+ this.sensitivePropertyProvider = sensitivePropertyProvider;
if (this.properties == null) {
throw new IllegalStateException("NiFiRegistryProperties cannot be null");
@@ -233,7 +242,12 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
final Map<String, String> authorizerProperties = new HashMap<>();
for (final Prop property : properties) {
- authorizerProperties.put(property.getName(), property.getValue());
+ if (!StringUtils.isBlank(property.getEncryption())) {
+ String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+ authorizerProperties.put(property.getName(), decryptedValue);
+ } else {
+ authorizerProperties.put(property.getName(), property.getValue());
+ }
}
return new StandardAuthorizerConfigurationContext(identifier, authorizerProperties);
}
@@ -387,6 +401,24 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
}
}
+ private String decryptValue(String cipherText, String encryptionScheme) throws SensitivePropertyProtectionException {
+ if (sensitivePropertyProvider == null) {
+ throw new SensitivePropertyProtectionException("Sensitive Property Provider dependency was never wired, so protected" +
+ "properties cannot be decrypted. This usually indicates that a master key for this NiFi Registry was not " +
+ "detected and configured during the bootstrap startup sequence. Contact the system administrator.");
+ }
+
+ if (!sensitivePropertyProvider.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
+ throw new SensitivePropertyProtectionException("Identity Provider configuration XML was protected using " +
+ encryptionScheme +
+ ", but the configured Sensitive Property Provider supports " +
+ sensitivePropertyProvider.getIdentifierKey() +
+ ". Cannot configure this Identity Provider due to failing to decrypt protected configuration properties.");
+ }
+
+ return sensitivePropertyProvider.unprotect(cipherText);
+ }
+
/**
* @return a default Authorizer to use when running unsecurely with no authorizer configured
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java
new file mode 100644
index 0000000..7859492
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java
@@ -0,0 +1,67 @@
+/*
+ * 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.registry.security.crypto;
+
+import org.apache.nifi.registry.properties.AESSensitivePropertyProvider;
+import org.apache.nifi.registry.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.registry.properties.SensitivePropertyProvider;
+import org.apache.nifi.registry.properties.SensitivePropertyProviderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+
+import javax.crypto.NoSuchPaddingException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+
+@Configuration
+public class SensitivePropertyProviderConfiguration implements SensitivePropertyProviderFactory {
+
+ private static final Logger logger = LoggerFactory.getLogger(SensitivePropertyProviderConfiguration.class);
+
+ @Autowired(required = false)
+ private CryptoKeyProvider masterKeyProvider;
+
+ /**
+ * @return a SensitivePropertyProvider initialized with the master key if present,
+ * or null if the master key is not present.
+ */
+ @Bean
+ @Override
+ public SensitivePropertyProvider getProvider() {
+ if (masterKeyProvider == null || masterKeyProvider.isEmpty()) {
+ // This NiFi Registry was not configured with a master key, so the assumption is
+ // the optional Spring bean normally provided by this method will never be needed
+ return null;
+ }
+
+ try {
+ // Note, this bean is intentionally NOT a singleton because we want the
+ // returned provider, which has a copy of the sensitive master key material
+ // to be reaped when it goes out of scope in order to decrease the time
+ // key material is held in memory.
+ String key = masterKeyProvider.getKey();
+ return new AESSensitivePropertyProvider(masterKeyProvider.getKey());
+ } catch (MissingCryptoKeyException | NoSuchAlgorithmException | NoSuchProviderException | NoSuchPaddingException e) {
+ logger.warn("Error creating AES Sensitive Property Provider", e);
+ throw new SensitivePropertyProtectionException("Error creating AES Sensitive Property Provider", e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-framework/src/main/xsd/authorizers.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/xsd/authorizers.xsd b/nifi-registry-framework/src/main/xsd/authorizers.xsd
index 278ff09..ed2a293 100644
--- a/nifi-registry-framework/src/main/xsd/authorizers.xsd
+++ b/nifi-registry-framework/src/main/xsd/authorizers.xsd
@@ -50,6 +50,7 @@
<xs:simpleContent>
<xs:extension base="xs:string">
<xs:attribute name="name" type="xs:string"></xs:attribute>
+ <xs:attribute name="encryption" type="xs:string"/>
</xs:extension>
</xs:simpleContent>
</xs:complexType>
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java b/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
index 459a61b..25c72f4 100644
--- a/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
+++ b/nifi-registry-jetty/src/main/java/org/apache/nifi/registry/jetty/JettyServer.java
@@ -17,6 +17,7 @@
package org.apache.nifi.registry.jetty;
import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.crypto.CryptoKeyProvider;
import org.apache.nifi.registry.properties.NiFiRegistryProperties;
import org.eclipse.jetty.annotations.AnnotationConfiguration;
import org.eclipse.jetty.server.Connector;
@@ -72,17 +73,19 @@ public class JettyServer {
};
private final NiFiRegistryProperties properties;
+ private final CryptoKeyProvider masterKeyProvider;
private final Server server;
private WebAppContext webUiContext;
private WebAppContext webApiContext;
private WebAppContext webDocsContext;
- public JettyServer(final NiFiRegistryProperties properties) {
+ public JettyServer(final NiFiRegistryProperties properties, final CryptoKeyProvider cryptoKeyProvider) {
final QueuedThreadPool threadPool = new QueuedThreadPool(properties.getWebThreads());
threadPool.setName("NiFi Registry Web Server");
this.properties = properties;
+ this.masterKeyProvider = cryptoKeyProvider;
this.server = new Server(threadPool);
// enable the annotation based configuration to ensure the jsp container is initialized properly
@@ -235,7 +238,10 @@ public class JettyServer {
webUiContext = loadWar(webUiWar, "/nifi-registry");
webApiContext = loadWar(webApiWar, "/nifi-registry-api");
+ logger.info("Adding {} object to ServletContext with key 'nifi-registry.properties'", properties.getClass().getSimpleName());
webApiContext.setAttribute("nifi-registry.properties", properties);
+ logger.info("Adding {} object to ServletContext with key 'nifi-registry.key'", masterKeyProvider.getClass().getSimpleName());
+ webApiContext.setAttribute("nifi-registry.key", masterKeyProvider);
// there is an issue scanning the asm repackaged jar so narrow down what we are scanning
webApiContext.setAttribute("org.eclipse.jetty.server.webapp.WebInfIncludeJarPattern", ".*/spring-[^/]*\\.jar$");
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/pom.xml b/nifi-registry-properties/pom.xml
index 24c9aa3..7e91d93 100644
--- a/nifi-registry-properties/pom.xml
+++ b/nifi-registry-properties/pom.xml
@@ -22,10 +22,52 @@
</parent>
<artifactId>nifi-registry-properties</artifactId>
<packaging>jar</packaging>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.gmavenplus</groupId>
+ <artifactId>gmavenplus-plugin</artifactId>
+ <version>1.5</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>addTestSources</goal>
+ <goal>testCompile</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
<dependencies>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.bouncycastle</groupId>
+ <artifactId>bcprov-jdk15on</artifactId>
+ <version>1.55</version>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.groovy</groupId>
+ <artifactId>groovy-all</artifactId>
+ <version>2.4.12</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>cglib</groupId>
+ <artifactId>cglib-nodep</artifactId>
+ <version>2.2.2</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-simple</artifactId>
+ <version>1.7.12</version>
+ <scope>test</scope>
+ </dependency>
</dependencies>
</project>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProvider.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProvider.java
new file mode 100644
index 0000000..b7d1d2e
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProvider.java
@@ -0,0 +1,265 @@
+/*
+ * 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.registry.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+import org.bouncycastle.util.encoders.Hex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.BadPaddingException;
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.StandardCharsets;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.SecureRandom;
+import java.security.Security;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class AESSensitivePropertyProvider implements SensitivePropertyProvider {
+ private static final Logger logger = LoggerFactory.getLogger(AESSensitivePropertyProvider.class);
+
+ private static final String IMPLEMENTATION_NAME = "AES Sensitive Property Provider";
+ private static final String IMPLEMENTATION_KEY = "aes/gcm/";
+ private static final String ALGORITHM = "AES/GCM/NoPadding";
+ private static final String PROVIDER = "BC";
+ private static final String DELIMITER = "||"; // "|" is not a valid Base64 character, so ensured not to be present in cipher text
+ private static final int IV_LENGTH = 12;
+ private static final int MIN_CIPHER_TEXT_LENGTH = IV_LENGTH * 4 / 3 + DELIMITER.length() + 1;
+
+ private Cipher cipher;
+ private final SecretKey key;
+
+ public AESSensitivePropertyProvider(String keyHex) throws NoSuchPaddingException, NoSuchAlgorithmException, NoSuchProviderException {
+ byte[] key = validateKey(keyHex);
+
+ try {
+ Security.addProvider(new BouncyCastleProvider());
+ cipher = Cipher.getInstance(ALGORITHM, PROVIDER);
+ // Only store the key if the cipher was initialized successfully
+ this.key = new SecretKeySpec(key, "AES");
+ } catch (NoSuchAlgorithmException | NoSuchProviderException | NoSuchPaddingException e) {
+ logger.error("Encountered an error initializing the {}: {}", IMPLEMENTATION_NAME, e.getMessage());
+ throw new SensitivePropertyProtectionException("Error initializing the protection cipher", e);
+ }
+ }
+
+ private byte[] validateKey(String keyHex) {
+ if (keyHex == null || StringUtils.isBlank(keyHex)) {
+ throw new SensitivePropertyProtectionException("The key cannot be empty");
+ }
+ keyHex = formatHexKey(keyHex);
+ if (!isHexKeyValid(keyHex)) {
+ throw new SensitivePropertyProtectionException("The key must be a valid hexadecimal key");
+ }
+ byte[] key = Hex.decode(keyHex);
+ final List<Integer> validKeyLengths = getValidKeyLengths();
+ if (!validKeyLengths.contains(key.length * 8)) {
+ List<String> validKeyLengthsAsStrings = validKeyLengths.stream().map(i -> Integer.toString(i)).collect(Collectors.toList());
+ throw new SensitivePropertyProtectionException("The key (" + key.length * 8 + " bits) must be a valid length: " + StringUtils.join(validKeyLengthsAsStrings, ", "));
+ }
+ return key;
+ }
+
+ public AESSensitivePropertyProvider(byte[] key) throws NoSuchPaddingException, NoSuchAlgorithmException, NoSuchProviderException {
+ this(key == null ? "" : Hex.toHexString(key));
+ }
+
+ private static String formatHexKey(String input) {
+ if (input == null || StringUtils.isBlank(input)) {
+ return "";
+ }
+ return input.replaceAll("[^0-9a-fA-F]", "").toLowerCase();
+ }
+
+ private static boolean isHexKeyValid(String key) {
+ if (key == null || StringUtils.isBlank(key)) {
+ return false;
+ }
+ // Key length is in "nibbles" (i.e. one hex char = 4 bits)
+ return getValidKeyLengths().contains(key.length() * 4) && key.matches("^[0-9a-fA-F]*$");
+ }
+
+ private static List<Integer> getValidKeyLengths() {
+ List<Integer> validLengths = new ArrayList<>();
+ validLengths.add(128);
+
+ try {
+ if (Cipher.getMaxAllowedKeyLength("AES") > 128) {
+ validLengths.add(192);
+ validLengths.add(256);
+ } else {
+ logger.warn("JCE Unlimited Strength Cryptography Jurisdiction policies are not available, so the max key length is 128 bits");
+ }
+ } catch (NoSuchAlgorithmException e) {
+ logger.warn("Encountered an error determining the max key length", e);
+ }
+
+ return validLengths;
+ }
+
+ /**
+ * Returns the name of the underlying implementation.
+ *
+ * @return the name of this sensitive property provider
+ */
+ @Override
+ public String getName() {
+ return IMPLEMENTATION_NAME;
+ }
+
+ /**
+ * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+ *
+ * @return the key to persist in the sibling property
+ */
+ @Override
+ public String getIdentifierKey() {
+ return IMPLEMENTATION_KEY + getKeySize(Hex.toHexString(key.getEncoded()));
+ }
+
+ private int getKeySize(String key) {
+ if (StringUtils.isBlank(key)) {
+ return 0;
+ } else {
+ // A key in hexadecimal format has one char per nibble (4 bits)
+ return formatHexKey(key).length() * 4;
+ }
+ }
+
+ /**
+ * Returns the encrypted cipher text.
+ *
+ * @param unprotectedValue the sensitive value
+ * @return the value to persist in the {@code nifi.properties} file
+ * @throws SensitivePropertyProtectionException if there is an exception encrypting the value
+ */
+ @Override
+ public String protect(String unprotectedValue) throws SensitivePropertyProtectionException {
+ if (unprotectedValue == null || unprotectedValue.trim().length() == 0) {
+ throw new IllegalArgumentException("Cannot encrypt an empty value");
+ }
+
+ // Generate IV
+ byte[] iv = generateIV();
+ if (iv.length < IV_LENGTH) {
+ throw new IllegalArgumentException("The IV (" + iv.length + " bytes) must be at least " + IV_LENGTH + " bytes");
+ }
+
+ try {
+ // Initialize cipher for encryption
+ cipher.init(Cipher.ENCRYPT_MODE, this.key, new IvParameterSpec(iv));
+
+ byte[] plainBytes = unprotectedValue.getBytes(StandardCharsets.UTF_8);
+ byte[] cipherBytes = cipher.doFinal(plainBytes);
+ logger.info(getName() + " encrypted a sensitive value successfully");
+ return base64Encode(iv) + DELIMITER + base64Encode(cipherBytes);
+ // return Base64.toBase64String(iv) + DELIMITER + Base64.toBase64String(cipherBytes);
+ } catch (BadPaddingException | IllegalBlockSizeException | EncoderException | InvalidAlgorithmParameterException | InvalidKeyException e) {
+ final String msg = "Error encrypting a protected value";
+ logger.error(msg, e);
+ throw new SensitivePropertyProtectionException(msg, e);
+ }
+ }
+
+ private String base64Encode(byte[] input) {
+ return Base64.toBase64String(input).replaceAll("=", "");
+ }
+
+ /**
+ * Generates a new random IV of 12 bytes using {@link SecureRandom}.
+ *
+ * @return the IV
+ */
+ private byte[] generateIV() {
+ byte[] iv = new byte[IV_LENGTH];
+ new SecureRandom().nextBytes(iv);
+ return iv;
+ }
+
+ /**
+ * Returns the decrypted plaintext.
+ *
+ * @param protectedValue the cipher text read from the {@code nifi.properties} file
+ * @return the raw value to be used by the application
+ * @throws SensitivePropertyProtectionException if there is an error decrypting the cipher text
+ */
+ @Override
+ public String unprotect(String protectedValue) throws SensitivePropertyProtectionException {
+ if (protectedValue == null || protectedValue.trim().length() < MIN_CIPHER_TEXT_LENGTH) {
+ throw new IllegalArgumentException("Cannot decrypt a cipher text shorter than " + MIN_CIPHER_TEXT_LENGTH + " chars");
+ }
+
+ if (!protectedValue.contains(DELIMITER)) {
+ throw new IllegalArgumentException("The cipher text does not contain the delimiter " + DELIMITER + " -- it should be of the form Base64(IV) || Base64(cipherText)");
+ }
+
+ protectedValue = protectedValue.trim();
+
+ final String IV_B64 = protectedValue.substring(0, protectedValue.indexOf(DELIMITER));
+ byte[] iv = Base64.decode(IV_B64);
+ if (iv.length < IV_LENGTH) {
+ throw new IllegalArgumentException("The IV (" + iv.length + " bytes) must be at least " + IV_LENGTH + " bytes");
+ }
+
+ String CIPHERTEXT_B64 = protectedValue.substring(protectedValue.indexOf(DELIMITER) + 2);
+
+ // Restore the = padding if necessary to reconstitute the GCM MAC check
+ if (CIPHERTEXT_B64.length() % 4 != 0) {
+ final int paddedLength = CIPHERTEXT_B64.length() + 4 - (CIPHERTEXT_B64.length() % 4);
+ CIPHERTEXT_B64 = StringUtils.rightPad(CIPHERTEXT_B64, paddedLength, '=');
+ }
+
+ try {
+ byte[] cipherBytes = Base64.decode(CIPHERTEXT_B64);
+
+ cipher.init(Cipher.DECRYPT_MODE, this.key, new IvParameterSpec(iv));
+ byte[] plainBytes = cipher.doFinal(cipherBytes);
+ logger.debug(getName() + " decrypted a sensitive value successfully");
+ return new String(plainBytes, StandardCharsets.UTF_8);
+ } catch (BadPaddingException | IllegalBlockSizeException | DecoderException | InvalidAlgorithmParameterException | InvalidKeyException e) {
+ final String msg = "Error decrypting a protected value";
+ logger.error(msg, e);
+ throw new SensitivePropertyProtectionException(msg, e);
+ }
+ }
+
+ public static int getIvLength() {
+ return IV_LENGTH;
+ }
+
+ public static int getMinCipherTextLength() {
+ return MIN_CIPHER_TEXT_LENGTH;
+ }
+
+ public static String getDelimiter() {
+ return DELIMITER;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProviderFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProviderFactory.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProviderFactory.java
new file mode 100644
index 0000000..5c24a73
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/AESSensitivePropertyProviderFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.registry.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.NoSuchPaddingException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+
+public class AESSensitivePropertyProviderFactory implements SensitivePropertyProviderFactory {
+ private static final Logger logger = LoggerFactory.getLogger(AESSensitivePropertyProviderFactory.class);
+
+ private String keyHex;
+
+ public AESSensitivePropertyProviderFactory(String keyHex) {
+ this.keyHex = keyHex;
+ }
+
+ public SensitivePropertyProvider getProvider() throws SensitivePropertyProtectionException {
+ try {
+ if (keyHex != null && !StringUtils.isBlank(keyHex)) {
+ return new AESSensitivePropertyProvider(keyHex);
+ } else {
+ throw new SensitivePropertyProtectionException("The provider factory cannot generate providers without a key");
+ }
+ } catch (NoSuchAlgorithmException | NoSuchProviderException | NoSuchPaddingException e) {
+ String msg = "Error creating AES Sensitive Property Provider";
+ logger.warn(msg, e);
+ throw new SensitivePropertyProtectionException(msg, e);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "SensitivePropertyProviderFactory for creating AESSensitivePropertyProviders";
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/MultipleSensitivePropertyProtectionException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/MultipleSensitivePropertyProtectionException.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/MultipleSensitivePropertyProtectionException.java
new file mode 100644
index 0000000..df4047f
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/MultipleSensitivePropertyProtectionException.java
@@ -0,0 +1,129 @@
+/*
+ * 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.registry.properties;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+public class MultipleSensitivePropertyProtectionException extends SensitivePropertyProtectionException {
+
+ private Set<String> failedKeys;
+
+ /**
+ * Constructs a new throwable with {@code null} as its detail message.
+ * The cause is not initialized, and may subsequently be initialized by a
+ * call to {@link #initCause}.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ */
+ public MultipleSensitivePropertyProtectionException() {
+ }
+
+ /**
+ * Constructs a new throwable with the specified detail message. The
+ * cause is not initialized, and may subsequently be initialized by
+ * a call to {@link #initCause}.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param message the detail message. The detail message is saved for
+ * later retrieval by the {@link #getMessage()} method.
+ */
+ public MultipleSensitivePropertyProtectionException(String message) {
+ super(message);
+ }
+
+ /**
+ * Constructs a new throwable with the specified detail message and
+ * cause. <p>Note that the detail message associated with
+ * {@code cause} is <i>not</i> automatically incorporated in
+ * this throwable's detail message.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param message the detail message (which is saved for later retrieval
+ * by the {@link #getMessage()} method).
+ * @param cause the cause (which is saved for later retrieval by the
+ * {@link #getCause()} method). (A {@code null} value is
+ * permitted, and indicates that the cause is nonexistent or
+ * unknown.)
+ * @since 1.4
+ */
+ public MultipleSensitivePropertyProtectionException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ /**
+ * Constructs a new throwable with the specified cause and a detail
+ * message of {@code (cause==null ? null : cause.toString())} (which
+ * typically contains the class and detail message of {@code cause}).
+ * This constructor is useful for throwables that are little more than
+ * wrappers for other throwables (for example, PrivilegedActionException).
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param cause the cause (which is saved for later retrieval by the
+ * {@link #getCause()} method). (A {@code null} value is
+ * permitted, and indicates that the cause is nonexistent or
+ * unknown.)
+ * @since 1.4
+ */
+ public MultipleSensitivePropertyProtectionException(Throwable cause) {
+ super(cause);
+ }
+
+ /**
+ * Constructs a new exception with the provided message and a unique set of the keys that caused the error.
+ *
+ * @param message the message
+ * @param failedKeys any failed keys
+ */
+ public MultipleSensitivePropertyProtectionException(String message, Collection<String> failedKeys) {
+ this(message, failedKeys, null);
+ }
+
+ /**
+ * Constructs a new exception with the provided message and a unique set of the keys that caused the error.
+ *
+ * @param message the message
+ * @param failedKeys any failed keys
+ * @param cause the cause (which is saved for later retrieval by the
+ * {@link #getCause()} method). (A {@code null} value is
+ * permitted, and indicates that the cause is nonexistent or
+ * unknown.)
+ */
+ public MultipleSensitivePropertyProtectionException(String message, Collection<String> failedKeys, Throwable cause) {
+ super(message, cause);
+ this.failedKeys = new HashSet<>(failedKeys);
+ }
+
+ public Set<String> getFailedKeys() {
+ return this.failedKeys;
+ }
+
+ @Override
+ public String toString() {
+ return "SensitivePropertyProtectionException for [" + StringUtils.join(this.failedKeys, ", ") + "]: " + getLocalizedMessage();
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryPropertiesLoader.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryPropertiesLoader.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryPropertiesLoader.java
new file mode 100644
index 0000000..5ceffd1
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryPropertiesLoader.java
@@ -0,0 +1,148 @@
+/*
+ * 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.registry.properties;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.Cipher;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+
+public class NiFiRegistryPropertiesLoader {
+
+ private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryPropertiesLoader.class);
+
+ private static final String RELATIVE_PATH = "conf/nifi-registry.properties";
+
+ private String keyHex;
+
+ // Future enhancement: allow for external registration of new providers
+ private static SensitivePropertyProviderFactory sensitivePropertyProviderFactory;
+
+ /**
+ * Returns an instance of the loader configured with the key.
+ * <p>
+ * <p>
+ * NOTE: This method is used reflectively by the process which starts NiFi
+ * so changes to it must be made in conjunction with that mechanism.</p>
+ *
+ * @param keyHex the key used to encrypt any sensitive properties
+ * @return the configured loader
+ */
+ public static NiFiRegistryPropertiesLoader withKey(String keyHex) {
+ NiFiRegistryPropertiesLoader loader = new NiFiRegistryPropertiesLoader();
+ loader.setKeyHex(keyHex);
+ return loader;
+ }
+
+ /**
+ * Sets the hexadecimal key used to unprotect properties encrypted with
+ * {@link AESSensitivePropertyProvider}. If the key has already been set,
+ * calling this method will throw a {@link RuntimeException}.
+ *
+ * @param keyHex the key in hexadecimal format
+ */
+ public void setKeyHex(String keyHex) {
+ if (this.keyHex == null || this.keyHex.trim().isEmpty()) {
+ this.keyHex = keyHex;
+ } else {
+ throw new RuntimeException("Cannot overwrite an existing key");
+ }
+ }
+
+ private static String getDefaultProviderKey() {
+ try {
+ return "aes/gcm/" + (Cipher.getMaxAllowedKeyLength("AES") > 128 ? "256" : "128");
+ } catch (NoSuchAlgorithmException e) {
+ return "aes/gcm/128";
+ }
+ }
+
+ private void initializeSensitivePropertyProviderFactory() {
+ sensitivePropertyProviderFactory = new AESSensitivePropertyProviderFactory(keyHex);
+ }
+
+ private SensitivePropertyProvider getSensitivePropertyProvider() {
+ initializeSensitivePropertyProviderFactory();
+ return sensitivePropertyProviderFactory.getProvider();
+ }
+
+ /**
+ * Returns a {@link ProtectedNiFiRegistryProperties} instance loaded from the
+ * serialized form in the file. Responsible for actually reading from disk
+ * and deserializing the properties. Returns a protected instance to allow
+ * for decryption operations.
+ *
+ * @param file the file containing serialized properties
+ * @return the ProtectedNiFiProperties instance
+ */
+ ProtectedNiFiRegistryProperties readProtectedPropertiesFromDisk(File file) {
+ if (file == null || !file.exists() || !file.canRead()) {
+ String path = (file == null ? "missing file" : file.getAbsolutePath());
+ logger.error("Cannot read from '{}' -- file is missing or not readable", path);
+ throw new IllegalArgumentException("NiFi Registry properties file missing or unreadable");
+ }
+
+ final NiFiRegistryProperties rawProperties = new NiFiRegistryProperties();
+ try (final FileReader reader = new FileReader(file)) {
+ rawProperties.load(reader);
+ logger.info("Loaded {} properties from {}", rawProperties.size(), file.getAbsolutePath());
+ ProtectedNiFiRegistryProperties protectedNiFiRegistryProperties = new ProtectedNiFiRegistryProperties(rawProperties);
+ return protectedNiFiRegistryProperties;
+ } catch (final IOException ioe) {
+ logger.error("Cannot load properties file due to " + ioe.getLocalizedMessage());
+ throw new RuntimeException("Cannot load properties file due to " + ioe.getLocalizedMessage(), ioe);
+ }
+ }
+
+ /**
+ * Returns an instance of {@link NiFiRegistryProperties} loaded from the provided
+ * {@link File}. If any properties are protected, will attempt to use the appropriate
+ * {@link SensitivePropertyProvider} to unprotect them transparently.
+ *
+ * @param file the File containing the serialized properties
+ * @return the NiFiProperties instance
+ */
+ public NiFiRegistryProperties load(File file) {
+ ProtectedNiFiRegistryProperties protectedNiFiRegistryProperties = readProtectedPropertiesFromDisk(file);
+ if (protectedNiFiRegistryProperties.hasProtectedKeys()) {
+ protectedNiFiRegistryProperties.addSensitivePropertyProvider(getSensitivePropertyProvider());
+ }
+
+ return protectedNiFiRegistryProperties.getUnprotectedProperties();
+ }
+
+ /**
+ * Returns an instance of {@link NiFiRegistryProperties}. The path must not be empty.
+ *
+ * @param path the path of the serialized properties file
+ * @return the NiFiRegistryProperties instance
+ * @see NiFiRegistryPropertiesLoader#load(File)
+ */
+ public NiFiRegistryProperties load(String path) {
+ if (path != null && !path.trim().isEmpty()) {
+ return load(new File(path));
+ } else {
+ logger.error("Cannot read from '{}' -- path is null or empty", path);
+ throw new IllegalArgumentException("NiFi Registry properties file path empty or null");
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/ProtectedNiFiRegistryProperties.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/ProtectedNiFiRegistryProperties.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/ProtectedNiFiRegistryProperties.java
new file mode 100644
index 0000000..5debc4a
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/ProtectedNiFiRegistryProperties.java
@@ -0,0 +1,528 @@
+/*
+ * 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.registry.properties;
+
+import org.apache.commons.lang3.StringUtils;
+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.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Wrapper class of {@link NiFiRegistryProperties} for intermediate phase when
+ * {@link NiFiRegistryPropertiesLoader} loads the raw properties file and performs
+ * unprotection activities before returning an instance of {@link NiFiRegistryProperties}.
+ */
+class ProtectedNiFiRegistryProperties {
+ private static final Logger logger = LoggerFactory.getLogger(ProtectedNiFiRegistryProperties.class);
+
+ private NiFiRegistryProperties properties;
+
+ private Map<String, SensitivePropertyProvider> localProviderCache = new HashMap<>();
+
+ // Additional "sensitive" property key
+ public static final String ADDITIONAL_SENSITIVE_PROPERTIES_KEY = "nifi.registry.sensitive.props.additional.keys";
+
+ // Default list of "sensitive" property keys
+ public static final List<String> DEFAULT_SENSITIVE_PROPERTIES = new ArrayList<>(asList(
+ NiFiRegistryProperties.SECURITY_KEY_PASSWD,
+ NiFiRegistryProperties.SECURITY_KEYSTORE_PASSWD,
+ NiFiRegistryProperties.SECURITY_TRUSTSTORE_PASSWD));
+
+ public ProtectedNiFiRegistryProperties() {
+ this(null);
+ }
+
+ /**
+ * Creates an instance containing the provided {@link NiFiRegistryProperties}.
+ *
+ * @param props the NiFiProperties to contain
+ */
+ public ProtectedNiFiRegistryProperties(NiFiRegistryProperties props) {
+ if (props == null) {
+ props = new NiFiRegistryProperties();
+ }
+ this.properties = props;
+ logger.debug("Loaded {} properties (including {} protection schemes) into ProtectedNiFiProperties",
+ getPropertyKeysIncludingProtectionSchemes().size(), getProtectedPropertyKeys().size());
+ }
+
+ /**
+ * 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
+ */
+ // @Override
+ public String getProperty(String key) {
+ return getInternalNiFiProperties().getProperty(key);
+ }
+
+ /**
+ * Returns the internal representation of the {@link NiFiRegistryProperties} -- protected
+ * or not as determined by the current state. No guarantee is made to the
+ * protection state of these properties. If the internal reference is null, a new
+ * {@link NiFiRegistryProperties} instance is created.
+ *
+ * @return the internal properties
+ */
+ NiFiRegistryProperties getInternalNiFiProperties() {
+ if (this.properties == null) {
+ this.properties = new NiFiRegistryProperties();
+ }
+
+ return this.properties;
+ }
+
+ /**
+ * Returns the number of properties in the NiFiRegistryProperties,
+ * 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() {
+ return getPropertyKeysExcludingProtectionSchemes().size();
+ }
+
+ /**
+ * Returns the complete set of property keys in the NiFiRegistryProperties,
+ * including any protection keys (i.e. 'x.y.z.protected').
+ *
+ * @return the set of property keys
+ */
+ Set<String> getPropertyKeysIncludingProtectionSchemes() {
+ return getInternalNiFiProperties().getPropertyKeys();
+ }
+
+ /**
+ * Returns the set of property keys in the NiFiRegistryProperties,
+ * excluding any protection keys (i.e. 'x.y.z.protected').
+ *
+ * @return the set of property keys
+ */
+ Set<String> getPropertyKeysExcludingProtectionSchemes() {
+ Set<String> filteredKeys = getPropertyKeysIncludingProtectionSchemes();
+ filteredKeys.removeIf(p -> p.endsWith(".protected"));
+ return filteredKeys;
+ }
+
+ /**
+ * 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.registry.property.1; nifi.registry.property.2, nifi.registry.property.3"
+ * @return a List containing the split and trimmed properties
+ */
+ private static List<String> splitMultipleProperties(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;
+ }
+ }
+
+ /**
+ * Returns a list of the keys identifying "sensitive" properties.
+ *
+ * There is a default list, and additional keys can be provided in the
+ * {@code nifi.registry.sensitive.props.additional.keys} property in {@code nifi-registry.properties}.
+ *
+ * @return the list of sensitive property keys
+ */
+ public List<String> getSensitivePropertyKeys() {
+ String additionalPropertiesString = getProperty(ADDITIONAL_SENSITIVE_PROPERTIES_KEY);
+ if (additionalPropertiesString == null || additionalPropertiesString.trim().isEmpty()) {
+ return DEFAULT_SENSITIVE_PROPERTIES;
+ } 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(ADDITIONAL_SENSITIVE_PROPERTIES_KEY)) {
+ logger.warn("The key '{}' contains itself. This is poor practice and should be removed", ADDITIONAL_SENSITIVE_PROPERTIES_KEY);
+ additionalProperties.remove(ADDITIONAL_SENSITIVE_PROPERTIES_KEY);
+ }
+ additionalProperties.addAll(DEFAULT_SENSITIVE_PROPERTIES);
+ return additionalProperties;
+ }
+ }
+
+ /**
+ * 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 {@code nifi.properties}.
+ *
+ * @return the list of sensitive property keys
+ */
+ public List<String> getPopulatedSensitivePropertyKeys() {
+ List<String> allSensitiveKeys = getSensitivePropertyKeys();
+ return allSensitiveKeys.stream().filter(k -> StringUtils.isNotBlank(getProperty(k))).collect(Collectors.toList());
+ }
+
+ /**
+ * Returns true if any sensitive keys are protected.
+ *
+ * @return true if any key is protected; false otherwise
+ */
+ public boolean hasProtectedKeys() {
+ List<String> sensitiveKeys = getSensitivePropertyKeys();
+ for (String k : sensitiveKeys) {
+ if (isPropertyProtected(k)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * 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
+ */
+ public Map<String, String> getProtectedPropertyKeys() {
+ List<String> sensitiveKeys = getSensitivePropertyKeys();
+
+ Map<String, String> traditionalProtectedProperties = new HashMap<>();
+ for (String key : sensitiveKeys) {
+ String protection = getProperty(getProtectionKey(key));
+ if (StringUtils.isNotBlank(protection) && StringUtils.isNotBlank(getProperty(key))) {
+ traditionalProtectedProperties.put(key, protection);
+ }
+ }
+
+ return traditionalProtectedProperties;
+ }
+
+ /**
+ * Returns the unique set of all protection schemes currently in use for this instance.
+ *
+ * @return the set of protection schemes
+ */
+ public Set<String> getProtectionSchemes() {
+ return new HashSet<>(getProtectedPropertyKeys().values());
+ }
+
+ /**
+ * 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
+ */
+ public int getPercentOfSensitivePropertiesProtected() {
+ return (int) Math.round(getProtectedPropertyKeys().size() / ((double) getPopulatedSensitivePropertyKeys().size()) * 100);
+ }
+
+ /**
+ * Returns true if the property identified by this key is considered sensitive in this instance of {@code NiFiProperties}.
+ * Some properties are sensitive by default, while others can be specified by
+ * {@link ProtectedNiFiRegistryProperties#ADDITIONAL_SENSITIVE_PROPERTIES_KEY}.
+ *
+ * @param key the key
+ * @return true if it is sensitive
+ * @see ProtectedNiFiRegistryProperties#getSensitivePropertyKeys()
+ */
+ public boolean isPropertySensitive(String key) {
+ // If the explicit check for ADDITIONAL_SENSITIVE_PROPERTIES_KEY is not here, this could loop infinitely
+ return key != null && !key.equals(ADDITIONAL_SENSITIVE_PROPERTIES_KEY) && 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 ProtectedNiFiRegistryProperties#getSensitivePropertyKeys()
+ */
+ public boolean isPropertyProtected(String key) {
+ return key != null && isPropertySensitive(key) && !StringUtils.isBlank(getProperty(getProtectionKey(key)));
+ }
+
+ /**
+ * Returns the sibling property key which specifies the protection scheme for this key.
+ * <p>
+ * Example:
+ * <p>
+ * nifi.registry.sensitive.key=ABCXYZ
+ * nifi.registry.sensitive.key.protected=aes/gcm/256
+ * <p>
+ * nifi.registry.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(String key) {
+ if (key == null || key.isEmpty()) {
+ throw new IllegalArgumentException("Cannot find protection key for null key");
+ }
+
+ return key + ".protected";
+ }
+
+ /**
+ * Returns the unprotected {@link NiFiRegistryProperties} instance. If none of the
+ * properties loaded are marked as protected, it will simply pass through the
+ * internal instance. If any are protected, it will drop the protection scheme keys
+ * and translate each protected value (encrypted, HSM-retrieved, etc.) into the raw
+ * value and store it under the original key.
+ * <p>
+ * If any property fails to unprotect, it will save that key and continue. After
+ * attempting all properties, it will throw an exception containing all failed
+ * properties. This is necessary because the order is not enforced, so all failed
+ * properties should be gathered together.
+ *
+ * @return the NiFiRegistryProperties instance with all raw values
+ * @throws SensitivePropertyProtectionException if there is a problem unprotecting one or more keys
+ */
+ public NiFiRegistryProperties getUnprotectedProperties() throws SensitivePropertyProtectionException {
+ if (hasProtectedKeys()) {
+ logger.debug("There are {} protected properties of {} sensitive properties ({}%)",
+ getProtectedPropertyKeys().size(),
+ getPopulatedSensitivePropertyKeys().size(),
+ getPercentOfSensitivePropertiesProtected());
+
+ NiFiRegistryProperties unprotectedProperties = new NiFiRegistryProperties();
+
+ Set<String> failedKeys = new HashSet<>();
+
+ for (String key : getPropertyKeysExcludingProtectionSchemes()) {
+ /* 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 {
+ unprotectedProperties.setProperty(key, unprotectValue(key, getProperty(key)));
+ } catch (SensitivePropertyProtectionException e) {
+ logger.warn("Failed to unprotect '{}'", key, e);
+ failedKeys.add(key);
+ }
+ } else {
+ unprotectedProperties.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());
+ }
+ }
+
+ return unprotectedProperties;
+ } else {
+ logger.debug("No protected properties");
+ return getInternalNiFiProperties();
+ }
+ }
+
+ /**
+ * Registers a new {@link SensitivePropertyProvider}. This method will throw a {@link UnsupportedOperationException} if a provider is already registered for the protection scheme.
+ *
+ * @param sensitivePropertyProvider the provider
+ */
+ void addSensitivePropertyProvider(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);
+ }
+
+ private String getDefaultProtectionScheme() {
+ if (!getSensitivePropertyProviders().isEmpty()) {
+ List<String> schemes = new ArrayList<>(getSensitivePropertyProviders().keySet());
+ Collections.sort(schemes);
+ return schemes.get(0);
+ } else {
+ throw new IllegalStateException("No registered protection schemes");
+ }
+ }
+
+ /**
+ * Returns a new instance of {@link NiFiRegistryProperties} with all populated sensitive values protected by the default protection scheme.
+ *
+ * Plain non-sensitive values are copied directly.
+ *
+ * @return the protected properties in a {@link NiFiRegistryProperties} object
+ * @throws IllegalStateException if no protection schemes are registered
+ */
+ NiFiRegistryProperties protectPlainProperties() {
+ try {
+ return protectPlainProperties(getDefaultProtectionScheme());
+ } catch (IllegalStateException e) {
+ final String msg = "Cannot protect properties with default scheme if no protection schemes are registered";
+ logger.warn(msg);
+ throw new IllegalStateException(msg, e);
+ }
+ }
+
+ /**
+ * Returns a new instance of {@link NiFiRegistryProperties} with all populated sensitive values protected by the provided protection scheme.
+ *
+ * Plain non-sensitive values are copied directly.
+ *
+ * @param protectionScheme the identifier key of the {@link SensitivePropertyProvider} to use
+ * @return the protected properties in a {@link NiFiRegistryProperties} object
+ */
+ NiFiRegistryProperties protectPlainProperties(String protectionScheme) {
+ SensitivePropertyProvider spp = getSensitivePropertyProvider(protectionScheme);
+
+ NiFiRegistryProperties protectedProperties = new NiFiRegistryProperties();
+
+ // Copy over the plain keys
+ Set<String> plainKeys = getPropertyKeysExcludingProtectionSchemes();
+ plainKeys.removeAll(getSensitivePropertyKeys());
+ for (String key : plainKeys) {
+ protectedProperties.setProperty(key, getInternalNiFiProperties().getProperty(key));
+ }
+
+ // Add the protected keys and the protection schemes
+ for (String key : getSensitivePropertyKeys()) {
+ final String plainValue = getProperty(key);
+ if (plainValue != null && !plainValue.trim().isEmpty()) {
+ final String protectedValue = spp.protect(plainValue);
+ protectedProperties.setProperty(key, protectedValue);
+ protectedProperties.setProperty(getProtectionKey(key), protectionScheme);
+ }
+ }
+
+ return protectedProperties;
+ }
+
+ /**
+ * Returns the number of properties that are marked as protected in the provided {@link NiFiRegistryProperties} instance
+ * without requiring external creation of a {@link ProtectedNiFiRegistryProperties} instance.
+ *
+ * @param plainProperties the instance to count protected properties
+ * @return the number of protected properties
+ */
+ public static int countProtectedProperties(NiFiRegistryProperties plainProperties) {
+ return new ProtectedNiFiRegistryProperties(plainProperties).getProtectedPropertyKeys().size();
+ }
+
+ /**
+ * Returns the number of properties that are marked as sensitive in the provided {@link NiFiRegistryProperties} instance
+ * without requiring external creation of a {@link ProtectedNiFiRegistryProperties} instance.
+ *
+ * @param plainProperties the instance to count sensitive properties
+ * @return the number of sensitive properties
+ */
+ public static int countSensitiveProperties(NiFiRegistryProperties plainProperties) {
+ return new ProtectedNiFiRegistryProperties(plainProperties).getSensitivePropertyKeys().size();
+ }
+
+ @Override
+ public String toString() {
+ final Set<String> providers = getSensitivePropertyProviders().keySet();
+ return new StringBuilder("ProtectedNiFiProperties instance with ")
+ .append(getPropertyKeysIncludingProtectionSchemes().size())
+ .append(" properties (")
+ .append(getProtectedPropertyKeys().size())
+ .append(" protected) and ")
+ .append(providers.size())
+ .append(" sensitive property providers: ")
+ .append(StringUtils.join(providers, ", "))
+ .toString();
+ }
+
+ /**
+ * Returns the local provider cache (null-safe) as a Map of protection schemes -> implementations.
+ *
+ * @return the map
+ */
+ private Map<String, SensitivePropertyProvider> getSensitivePropertyProviders() {
+ if (localProviderCache == null) {
+ localProviderCache = new HashMap<>();
+ }
+
+ return localProviderCache;
+ }
+
+ private SensitivePropertyProvider getSensitivePropertyProvider(String protectionScheme) {
+ if (isProviderAvailable(protectionScheme)) {
+ return getSensitivePropertyProviders().get(protectionScheme);
+ } else {
+ throw new SensitivePropertyProtectionException("No provider available for " + protectionScheme);
+ }
+ }
+
+ private boolean isProviderAvailable(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(String key, 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 {
+ SensitivePropertyProvider sensitivePropertyProvider = getSensitivePropertyProvider(protectionScheme);
+ return sensitivePropertyProvider.unprotect(retrievedValue);
+ } catch (SensitivePropertyProtectionException e) {
+ throw new SensitivePropertyProtectionException("Error unprotecting value for " + key, e.getCause());
+ }
+ }
+ return retrievedValue;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProtectionException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProtectionException.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProtectionException.java
new file mode 100644
index 0000000..2ffa902
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProtectionException.java
@@ -0,0 +1,89 @@
+/*
+ * 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.registry.properties;
+
+public class SensitivePropertyProtectionException extends RuntimeException {
+ /**
+ * Constructs a new throwable with {@code null} as its detail message.
+ * The cause is not initialized, and may subsequently be initialized by a
+ * call to {@link #initCause}.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ */
+ public SensitivePropertyProtectionException() {
+ }
+
+ /**
+ * Constructs a new throwable with the specified detail message. The
+ * cause is not initialized, and may subsequently be initialized by
+ * a call to {@link #initCause}.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param message the detail message. The detail message is saved for
+ * later retrieval by the {@link #getMessage()} method.
+ */
+ public SensitivePropertyProtectionException(String message) {
+ super(message);
+ }
+
+ /**
+ * Constructs a new throwable with the specified detail message and
+ * cause. <p>Note that the detail message associated with
+ * {@code cause} is <i>not</i> automatically incorporated in
+ * this throwable's detail message.
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param message the detail message (which is saved for later retrieval
+ * by the {@link #getMessage()} method).
+ * @param cause the cause (which is saved for later retrieval by the
+ * {@link #getCause()} method). (A {@code null} value is
+ * permitted, and indicates that the cause is nonexistent or
+ * unknown.)
+ */
+ public SensitivePropertyProtectionException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ /**
+ * Constructs a new throwable with the specified cause and a detail
+ * message of {@code (cause==null ? null : cause.toString())} (which
+ * typically contains the class and detail message of {@code cause}).
+ * This constructor is useful for throwables that are little more than
+ * wrappers for other throwables (for example, PrivilegedActionException).
+ * <p>
+ * <p>The {@link #fillInStackTrace()} method is called to initialize
+ * the stack trace data in the newly created throwable.
+ *
+ * @param cause the cause (which is saved for later retrieval by the
+ * {@link #getCause()} method). (A {@code null} value is
+ * permitted, and indicates that the cause is nonexistent or
+ * unknown.)
+ */
+ public SensitivePropertyProtectionException(Throwable cause) {
+ super(cause);
+ }
+
+ @Override
+ public String toString() {
+ return "SensitivePropertyProtectionException: " + getLocalizedMessage();
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/64211451/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProvider.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProvider.java
new file mode 100644
index 0000000..c0dd43c
--- /dev/null
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/SensitivePropertyProvider.java
@@ -0,0 +1,52 @@
+/*
+ * 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.registry.properties;
+
+public interface SensitivePropertyProvider {
+
+ /**
+ * Returns the name of the underlying implementation.
+ *
+ * @return the name of this sensitive property provider
+ */
+ String getName();
+
+ /**
+ * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+ *
+ * @return the key to persist in the sibling property
+ */
+ String getIdentifierKey();
+
+ /**
+ * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+ * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+ *
+ * @param unprotectedValue the sensitive value
+ * @return the value to persist in the {@code nifi.properties} file
+ */
+ String protect(String unprotectedValue) throws SensitivePropertyProtectionException;
+
+ /**
+ * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+ * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+ *
+ * @param protectedValue the protected value read from the {@code nifi.properties} file
+ * @return the raw value to be used by the application
+ */
+ String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+}
|