mirror of
https://github.com/apache/nifi.git
synced 2025-02-20 17:36:50 +00:00
NIFI-7621 & NIFI-7614 Updated terminology throughout codebase and documentation.
Fixed unit test failures introduced from static imports during refactoring. Signed-off-by: Joe Witt <joe.witt@gmail.com>
This commit is contained in:
parent
4f11e36260
commit
0fa8776f4d
2
.github/PULL_REQUEST_TEMPLATE.md
vendored
2
.github/PULL_REQUEST_TEMPLATE.md
vendored
@ -15,7 +15,7 @@ to ensure the following steps have been taken:
|
||||
|
||||
- [ ] 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.
|
||||
|
||||
- [ ] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
|
||||
- [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
|
||||
|
||||
- [ ] 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._
|
||||
|
||||
|
@ -46,7 +46,7 @@ Apache NiFi was made for dataflow. It supports highly configurable directed grap
|
||||
- Flows can be modified at runtime
|
||||
- Back pressure
|
||||
- Scales up to leverage full machine capability
|
||||
- Scales out with zero-master clustering model
|
||||
- Scales out with zero-leader clustering model
|
||||
- Data Provenance
|
||||
- Track dataflow from beginning to end
|
||||
- Designed for extension
|
||||
|
@ -227,22 +227,22 @@ public class CryptoUtils {
|
||||
|
||||
/**
|
||||
* Returns a map containing the key IDs and the parsed key from a key provider definition file.
|
||||
* The values in the file are decrypted using the master key provided. If the file is missing or empty,
|
||||
* The values in the file are decrypted using the root key provided. If the file is missing or empty,
|
||||
* cannot be read, or if no valid keys are read, a {@link KeyManagementException} will be thrown.
|
||||
*
|
||||
* @param filepath the key definition file path
|
||||
* @param masterKey the master key used to decrypt each key definition
|
||||
* @param rootKey the root key used to decrypt each key definition
|
||||
* @return a Map of key IDs to SecretKeys
|
||||
* @throws KeyManagementException if the file is missing or invalid
|
||||
*/
|
||||
public static Map<String, SecretKey> readKeys(String filepath, SecretKey masterKey) throws KeyManagementException {
|
||||
public static Map<String, SecretKey> readKeys(String filepath, SecretKey rootKey) throws KeyManagementException {
|
||||
Map<String, SecretKey> keys = new HashMap<>();
|
||||
|
||||
if (StringUtils.isBlank(filepath)) {
|
||||
throw new KeyManagementException("The key provider file is not present and readable");
|
||||
}
|
||||
if (masterKey == null) {
|
||||
throw new KeyManagementException("The master key must be provided to decrypt the individual keys");
|
||||
if (rootKey == null) {
|
||||
throw new KeyManagementException("The root key must be provided to decrypt the individual keys");
|
||||
}
|
||||
|
||||
File file = new File(filepath);
|
||||
@ -251,7 +251,7 @@ public class CryptoUtils {
|
||||
}
|
||||
|
||||
try (BufferedReader br = new BufferedReader(new FileReader(file))) {
|
||||
AESKeyedCipherProvider masterCipherProvider = new AESKeyedCipherProvider();
|
||||
AESKeyedCipherProvider rootCipherProvider = new AESKeyedCipherProvider();
|
||||
|
||||
String line;
|
||||
int l = 1;
|
||||
@ -266,13 +266,13 @@ public class CryptoUtils {
|
||||
byte[] base64Bytes = Base64.getDecoder().decode(components[1]);
|
||||
byte[] ivBytes = Arrays.copyOfRange(base64Bytes, 0, IV_LENGTH);
|
||||
|
||||
Cipher masterCipher = null;
|
||||
Cipher rootCipher = null;
|
||||
try {
|
||||
masterCipher = masterCipherProvider.getCipher(EncryptionMethod.AES_GCM, masterKey, ivBytes, false);
|
||||
rootCipher = rootCipherProvider.getCipher(EncryptionMethod.AES_GCM, rootKey, ivBytes, false);
|
||||
} catch (Exception e) {
|
||||
throw new KeyManagementException("Error building cipher to decrypt FileBaseKeyProvider definition at " + filepath, e);
|
||||
}
|
||||
byte[] individualKeyBytes = masterCipher.doFinal(Arrays.copyOfRange(base64Bytes, IV_LENGTH, base64Bytes.length));
|
||||
byte[] individualKeyBytes = rootCipher.doFinal(Arrays.copyOfRange(base64Bytes, IV_LENGTH, base64Bytes.length));
|
||||
|
||||
SecretKey key = new SecretKeySpec(individualKeyBytes, "AES");
|
||||
logger.debug("Read and decrypted key for " + keyId);
|
||||
@ -302,16 +302,16 @@ public class CryptoUtils {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the master key from the {@code bootstrap.conf} file used to encrypt various sensitive properties and data encryption keys.
|
||||
* Returns the root key from the {@code bootstrap.conf} file used to encrypt various sensitive properties and data encryption keys.
|
||||
*
|
||||
* @return the master key
|
||||
* @return the root key
|
||||
* @throws KeyManagementException if the key cannot be read
|
||||
*/
|
||||
public static SecretKey getMasterKey() throws KeyManagementException {
|
||||
public static SecretKey getRootKey() throws KeyManagementException {
|
||||
try {
|
||||
// Get the master encryption key from bootstrap.conf
|
||||
String masterKeyHex = extractKeyFromBootstrapFile();
|
||||
return new SecretKeySpec(Hex.decode(masterKeyHex), "AES");
|
||||
// Get the root encryption key from bootstrap.conf
|
||||
String rootKeyHex = extractKeyFromBootstrapFile();
|
||||
return new SecretKeySpec(Hex.decode(rootKeyHex), "AES");
|
||||
} catch (IOException e) {
|
||||
logger.error("Encountered an error: ", e);
|
||||
throw new KeyManagementException(e);
|
||||
|
@ -25,10 +25,10 @@ import org.slf4j.LoggerFactory;
|
||||
public class FileBasedKeyProvider extends StaticKeyProvider {
|
||||
private static final Logger logger = LoggerFactory.getLogger(FileBasedKeyProvider.class);
|
||||
|
||||
private String filepath;
|
||||
private final String filepath;
|
||||
|
||||
public FileBasedKeyProvider(String location, SecretKey masterKey) throws KeyManagementException {
|
||||
super(CryptoUtils.readKeys(location, masterKey));
|
||||
public FileBasedKeyProvider(String location, SecretKey rootKey) throws KeyManagementException {
|
||||
super(CryptoUtils.readKeys(location, rootKey));
|
||||
this.filepath = location;
|
||||
}
|
||||
|
||||
|
@ -33,16 +33,16 @@ public class KeyProviderFactory {
|
||||
/**
|
||||
* Returns a key provider instantiated from the configuration values in a {@link RepositoryEncryptionConfiguration} object.
|
||||
*
|
||||
* @param rec the data container for config values (usually extracted from {@link org.apache.nifi.util.NiFiProperties})
|
||||
* @param masterKey the master key used to decrypt wrapped keys
|
||||
* @param rec the data container for config values (usually extracted from {@link org.apache.nifi.util.NiFiProperties})
|
||||
* @param rootKey the root key used to decrypt wrapped keys
|
||||
* @return the configured key provider
|
||||
* @throws KeyManagementException if the key provider cannot be instantiated
|
||||
*/
|
||||
public static KeyProvider buildKeyProvider(RepositoryEncryptionConfiguration rec, SecretKey masterKey) throws KeyManagementException {
|
||||
public static KeyProvider buildKeyProvider(RepositoryEncryptionConfiguration rec, SecretKey rootKey) throws KeyManagementException {
|
||||
if (rec == null) {
|
||||
throw new KeyManagementException("The repository encryption configuration values are required to build a key provider");
|
||||
}
|
||||
return buildKeyProvider(rec.getKeyProviderImplementation(), rec.getKeyProviderLocation(), rec.getEncryptionKeyId(), rec.getEncryptionKeys(), masterKey);
|
||||
return buildKeyProvider(rec.getKeyProviderImplementation(), rec.getKeyProviderLocation(), rec.getEncryptionKeyId(), rec.getEncryptionKeys(), rootKey);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -52,12 +52,12 @@ public class KeyProviderFactory {
|
||||
* @param keyProviderLocation the filepath/URL of the stored keys
|
||||
* @param keyId the active key id
|
||||
* @param encryptionKeys the available encryption keys
|
||||
* @param masterKey the master key used to decrypt wrapped keys
|
||||
* @param rootKey the root key used to decrypt wrapped keys
|
||||
* @return the configured key provider
|
||||
* @throws KeyManagementException if the key provider cannot be instantiated
|
||||
*/
|
||||
public static KeyProvider buildKeyProvider(String implementationClassName, String keyProviderLocation, String keyId, Map<String, String> encryptionKeys,
|
||||
SecretKey masterKey) throws KeyManagementException {
|
||||
SecretKey rootKey) throws KeyManagementException {
|
||||
KeyProvider keyProvider;
|
||||
|
||||
implementationClassName = CryptoUtils.handleLegacyPackages(implementationClassName);
|
||||
@ -84,7 +84,7 @@ public class KeyProviderFactory {
|
||||
throw new KeyManagementException(msg);
|
||||
}
|
||||
} else if (FileBasedKeyProvider.class.getName().equals(implementationClassName)) {
|
||||
keyProvider = new FileBasedKeyProvider(keyProviderLocation, masterKey);
|
||||
keyProvider = new FileBasedKeyProvider(keyProviderLocation, rootKey);
|
||||
if (!keyProvider.keyExists(keyId)) {
|
||||
throw new KeyManagementException("The specified key ID " + keyId + " is not in the key definition file");
|
||||
}
|
||||
@ -96,13 +96,13 @@ public class KeyProviderFactory {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if this {@link KeyProvider} implementation requires the presence of the {@code master key} in order to decrypt the available data encryption keys.
|
||||
* Returns true if this {@link KeyProvider} implementation requires the presence of the {@code root key} in order to decrypt the available data encryption keys.
|
||||
*
|
||||
* @param implementationClassName the key provider implementation class
|
||||
* @return true if this implementation requires the master key to operate
|
||||
* @return true if this implementation requires the root key to operate
|
||||
* @throws KeyManagementException if the provided class name is not a valid key provider implementation
|
||||
*/
|
||||
public static boolean requiresMasterKey(String implementationClassName) throws KeyManagementException {
|
||||
public static boolean requiresRootKey(String implementationClassName) throws KeyManagementException {
|
||||
implementationClassName = CryptoUtils.handleLegacyPackages(implementationClassName);
|
||||
return FileBasedKeyProvider.class.getName().equals(implementationClassName);
|
||||
}
|
||||
|
@ -204,7 +204,7 @@ public class RepositoryEncryptorUtils {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a configured {@link KeyProvider} instance that does not require a {@code master key} to use (usually a {@link org.apache.nifi.security.kms.StaticKeyProvider}).
|
||||
* Returns a configured {@link KeyProvider} instance that does not require a {@code root key} to use (usually a {@link org.apache.nifi.security.kms.StaticKeyProvider}).
|
||||
*
|
||||
* @param niFiProperties the {@link NiFiProperties} object
|
||||
* @param repositoryType the {@link RepositoryType} indicator
|
||||
@ -216,38 +216,38 @@ public class RepositoryEncryptorUtils {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a configured {@link KeyProvider} instance that requires a {@code master key} to use
|
||||
* Returns a configured {@link KeyProvider} instance that requires a {@code root key} to use
|
||||
* (usually a {@link org.apache.nifi.security.kms.FileBasedKeyProvider} or an encrypted
|
||||
* {@link org.apache.nifi.security.kms.StaticKeyProvider}).
|
||||
*
|
||||
* @param niFiProperties the {@link NiFiProperties} object
|
||||
* @param masterKey the master encryption key used to encrypt the data encryption keys in the key provider configuration
|
||||
* @param rootKey the root encryption key used to encrypt the data encryption keys in the key provider configuration
|
||||
* @param repositoryType the {@link RepositoryType} indicator
|
||||
* @return the configured KeyProvider
|
||||
* @throws KeyManagementException if there is a problem with the configuration
|
||||
*/
|
||||
public static KeyProvider buildKeyProvider(NiFiProperties niFiProperties, SecretKey masterKey, RepositoryType repositoryType) throws KeyManagementException {
|
||||
public static KeyProvider buildKeyProvider(NiFiProperties niFiProperties, SecretKey rootKey, RepositoryType repositoryType) throws KeyManagementException {
|
||||
RepositoryEncryptionConfiguration rec = RepositoryEncryptionConfiguration.fromNiFiProperties(niFiProperties, repositoryType);
|
||||
|
||||
return buildKeyProviderFromConfig(masterKey, rec);
|
||||
return buildKeyProviderFromConfig(rootKey, rec);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a configured {@link KeyProvider} instance given the {@link RepositoryEncryptionConfiguration}.
|
||||
*
|
||||
* @param masterKey the master encryption key used to encrypt the data encryption keys in the key provider configuration
|
||||
* @param rootKey the root encryption key used to encrypt the data encryption keys in the key provider configuration
|
||||
* @param rec the repository-specific encryption configuration
|
||||
* @return the configured KeyProvider
|
||||
* @throws KeyManagementException if there is a problem with the configuration
|
||||
*/
|
||||
public static KeyProvider buildKeyProviderFromConfig(SecretKey masterKey, RepositoryEncryptionConfiguration rec) throws KeyManagementException {
|
||||
public static KeyProvider buildKeyProviderFromConfig(SecretKey rootKey, RepositoryEncryptionConfiguration rec) throws KeyManagementException {
|
||||
if (rec.getKeyProviderImplementation() == null) {
|
||||
final String keyProviderImplementationClass = determineKeyProviderImplementationClassName(rec.getRepositoryType());
|
||||
throw new KeyManagementException("Cannot create key provider because the NiFi properties are missing the following property: "
|
||||
+ keyProviderImplementationClass);
|
||||
}
|
||||
|
||||
return KeyProviderFactory.buildKeyProvider(rec, masterKey);
|
||||
return KeyProviderFactory.buildKeyProvider(rec, rootKey);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -289,9 +289,9 @@ public class RepositoryEncryptorUtils {
|
||||
try {
|
||||
KeyProvider keyProvider;
|
||||
final String keyProviderImplementation = niFiProperties.getProperty(determineKeyProviderImplementationClassName(repositoryType));
|
||||
if (KeyProviderFactory.requiresMasterKey(keyProviderImplementation)) {
|
||||
SecretKey masterKey = CryptoUtils.getMasterKey();
|
||||
keyProvider = buildKeyProvider(niFiProperties, masterKey, repositoryType);
|
||||
if (KeyProviderFactory.requiresRootKey(keyProviderImplementation)) {
|
||||
SecretKey rootKey = CryptoUtils.getRootKey();
|
||||
keyProvider = buildKeyProvider(niFiProperties, rootKey, repositoryType);
|
||||
} else {
|
||||
keyProvider = buildKeyProvider(niFiProperties, repositoryType);
|
||||
}
|
||||
@ -316,8 +316,8 @@ public class RepositoryEncryptorUtils {
|
||||
public static KeyProvider validateAndBuildRepositoryKeyProvider(RepositoryEncryptionConfiguration repositoryEncryptionConfiguration) throws IOException {
|
||||
// Initialize the encryption-specific fields
|
||||
try {
|
||||
SecretKey masterKey = KeyProviderFactory.requiresMasterKey(repositoryEncryptionConfiguration.getKeyProviderImplementation()) ? CryptoUtils.getMasterKey() : null;
|
||||
return buildKeyProviderFromConfig(masterKey, repositoryEncryptionConfiguration);
|
||||
SecretKey rootKey = KeyProviderFactory.requiresRootKey(repositoryEncryptionConfiguration.getKeyProviderImplementation()) ? CryptoUtils.getRootKey() : null;
|
||||
return buildKeyProviderFromConfig(rootKey, repositoryEncryptionConfiguration);
|
||||
} catch (KeyManagementException e) {
|
||||
String msg = "Encountered an error building the key provider";
|
||||
logger.error(msg, e);
|
||||
|
@ -71,7 +71,7 @@ public abstract class RepositoryEncryptionConfiguration {
|
||||
* Returns a map of all available encryption keys indexed by the key id if using
|
||||
* {@link StaticKeyProvider}. For
|
||||
* {@link FileBasedKeyProvider}, this method will return an
|
||||
* empty map because the keys must be loaded using the {@code master key} to decrypt them
|
||||
* empty map because the keys must be loaded using the {@code root key} to decrypt them
|
||||
* via {@link CryptoUtils#readKeys(String, SecretKey)}.
|
||||
*
|
||||
* @return a map of key ids & keys
|
||||
|
@ -343,8 +343,8 @@ class CryptoUtilsTest {
|
||||
@Test
|
||||
void testShouldReadKeys() {
|
||||
// Arrange
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
String keyFileName = "keys.nkp"
|
||||
@ -352,7 +352,7 @@ class CryptoUtilsTest {
|
||||
final int KEY_COUNT = 5
|
||||
List<String> lines = []
|
||||
KEY_COUNT.times { int i ->
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(rootKey)}")
|
||||
}
|
||||
|
||||
keyFile.text = lines.join("\n")
|
||||
@ -360,7 +360,7 @@ class CryptoUtilsTest {
|
||||
logger.info("File contents: \n${keyFile.text}")
|
||||
|
||||
// Act
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
|
||||
|
||||
// Assert
|
||||
@ -370,8 +370,8 @@ class CryptoUtilsTest {
|
||||
@Test
|
||||
void testShouldReadKeysWithDuplicates() {
|
||||
// Arrange
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
String keyFileName = "keys.nkp"
|
||||
@ -379,17 +379,17 @@ class CryptoUtilsTest {
|
||||
final int KEY_COUNT = 3
|
||||
List<String> lines = []
|
||||
KEY_COUNT.times { int i ->
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(rootKey)}")
|
||||
}
|
||||
|
||||
lines.add("key3=${generateEncryptedKey(masterKey)}")
|
||||
lines.add("key3=${generateEncryptedKey(rootKey)}")
|
||||
|
||||
keyFile.text = lines.join("\n")
|
||||
|
||||
logger.info("File contents: \n${keyFile.text}")
|
||||
|
||||
// Act
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
|
||||
|
||||
// Assert
|
||||
@ -399,8 +399,8 @@ class CryptoUtilsTest {
|
||||
@Test
|
||||
void testShouldReadKeysWithSomeMalformed() {
|
||||
// Arrange
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
String keyFileName = "keys.nkp"
|
||||
@ -408,12 +408,12 @@ class CryptoUtilsTest {
|
||||
final int KEY_COUNT = 5
|
||||
List<String> lines = []
|
||||
KEY_COUNT.times { int i ->
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(masterKey)}")
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(rootKey)}")
|
||||
}
|
||||
|
||||
// Insert the malformed keys in the middle
|
||||
lines.add(2, "keyX1==${generateEncryptedKey(masterKey)}")
|
||||
lines.add(4, "=${generateEncryptedKey(masterKey)}")
|
||||
lines.add(2, "keyX1==${generateEncryptedKey(rootKey)}")
|
||||
lines.add(4, "=${generateEncryptedKey(rootKey)}")
|
||||
lines.add(6, "keyX3=non Base64-encoded data")
|
||||
|
||||
keyFile.text = lines.join("\n")
|
||||
@ -421,7 +421,7 @@ class CryptoUtilsTest {
|
||||
logger.info("File contents: \n${keyFile.text}")
|
||||
|
||||
// Act
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
|
||||
|
||||
// Assert
|
||||
@ -431,8 +431,8 @@ class CryptoUtilsTest {
|
||||
@Test
|
||||
void testShouldNotReadKeysIfAllMalformed() {
|
||||
// Arrange
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
String keyFileName = "keys.nkp"
|
||||
@ -442,7 +442,7 @@ class CryptoUtilsTest {
|
||||
|
||||
// All of these keys are malformed
|
||||
KEY_COUNT.times { int i ->
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(masterKey)[0..<-4]}")
|
||||
lines.add("key${i + 1}=${generateEncryptedKey(rootKey)[0..<-4]}")
|
||||
}
|
||||
|
||||
keyFile.text = lines.join("\n")
|
||||
@ -451,7 +451,7 @@ class CryptoUtilsTest {
|
||||
|
||||
// Act
|
||||
def msg = shouldFail(KeyManagementException) {
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
|
||||
}
|
||||
|
||||
@ -462,8 +462,8 @@ class CryptoUtilsTest {
|
||||
@Test
|
||||
void testShouldNotReadKeysIfEmptyOrMissing() {
|
||||
// Arrange
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
String keyFileName = "empty.nkp"
|
||||
@ -472,13 +472,13 @@ class CryptoUtilsTest {
|
||||
|
||||
// Act
|
||||
def missingMsg = shouldFail(KeyManagementException) {
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(keyFile.path, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${keyFile.path}")
|
||||
}
|
||||
logger.expected("Missing file: ${missingMsg}")
|
||||
|
||||
def emptyMsg = shouldFail(KeyManagementException) {
|
||||
def readKeys = CryptoUtils.readKeys(null, masterKey)
|
||||
def readKeys = CryptoUtils.readKeys(null, rootKey)
|
||||
logger.info("Read ${readKeys.size()} keys from ${null}")
|
||||
}
|
||||
logger.expected("Empty file: ${emptyMsg}")
|
||||
@ -685,7 +685,7 @@ class CryptoUtilsTest {
|
||||
end - start
|
||||
}
|
||||
|
||||
private static String generateEncryptedKey(SecretKey masterKey) {
|
||||
private static String generateEncryptedKey(SecretKey rootKey) {
|
||||
byte[] ivBytes = new byte[16]
|
||||
byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]
|
||||
|
||||
@ -693,9 +693,9 @@ class CryptoUtilsTest {
|
||||
sr.nextBytes(ivBytes)
|
||||
sr.nextBytes(keyBytes)
|
||||
|
||||
Cipher masterCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
|
||||
masterCipher.init(Cipher.ENCRYPT_MODE, masterKey, new IvParameterSpec(ivBytes))
|
||||
byte[] cipherBytes = masterCipher.doFinal(keyBytes)
|
||||
Cipher rootCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
|
||||
rootCipher.init(Cipher.ENCRYPT_MODE, rootKey, new IvParameterSpec(ivBytes))
|
||||
byte[] cipherBytes = rootCipher.doFinal(keyBytes)
|
||||
|
||||
Base64.encoder.encodeToString(CryptoUtils.concatByteArrays(ivBytes, cipherBytes))
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ class KeyProviderFactoryTest {
|
||||
|
||||
private static final String ORIGINAL_PROPERTIES_PATH = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
|
||||
|
||||
private static final SecretKey MASTER_KEY = new SecretKeySpec(Hex.decode(KEY_HEX), "AES")
|
||||
private static final SecretKey ROOT_KEY = new SecretKeySpec(Hex.decode(KEY_HEX), "AES")
|
||||
|
||||
@ClassRule
|
||||
public static TemporaryFolder tempFolder = new TemporaryFolder()
|
||||
@ -97,21 +97,21 @@ class KeyProviderFactoryTest {
|
||||
}
|
||||
|
||||
private static void populateKeyDefinitionsFile(String path = "src/test/resources/conf/filebased.kp") {
|
||||
String masterKeyHex = KEY_HEX
|
||||
SecretKey masterKey = new SecretKeySpec(Hex.decode(masterKeyHex), "AES")
|
||||
String rootKeyHex = KEY_HEX
|
||||
SecretKey rootKey = new SecretKeySpec(Hex.decode(rootKeyHex), "AES")
|
||||
|
||||
// Generate the file
|
||||
File keyFile = new File(path)
|
||||
final int KEY_COUNT = 1
|
||||
List<String> lines = []
|
||||
KEY_COUNT.times { int i ->
|
||||
lines.add("K${i + 1}=${generateEncryptedKey(masterKey)}")
|
||||
lines.add("K${i + 1}=${generateEncryptedKey(rootKey)}")
|
||||
}
|
||||
|
||||
keyFile.text = lines.join("\n")
|
||||
}
|
||||
|
||||
private static String generateEncryptedKey(SecretKey masterKey) {
|
||||
private static String generateEncryptedKey(SecretKey rootKey) {
|
||||
byte[] ivBytes = new byte[16]
|
||||
byte[] keyBytes = new byte[isUnlimitedStrengthCryptoAvailable() ? 32 : 16]
|
||||
|
||||
@ -119,9 +119,9 @@ class KeyProviderFactoryTest {
|
||||
sr.nextBytes(ivBytes)
|
||||
sr.nextBytes(keyBytes)
|
||||
|
||||
Cipher masterCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
|
||||
masterCipher.init(Cipher.ENCRYPT_MODE, masterKey, new IvParameterSpec(ivBytes))
|
||||
byte[] cipherBytes = masterCipher.doFinal(keyBytes)
|
||||
Cipher rootCipher = Cipher.getInstance("AES/GCM/NoPadding", "BC")
|
||||
rootCipher.init(Cipher.ENCRYPT_MODE, rootKey, new IvParameterSpec(ivBytes))
|
||||
byte[] cipherBytes = rootCipher.doFinal(keyBytes)
|
||||
|
||||
Base64.encoder.encodeToString(CryptoUtils.concatByteArrays(ivBytes, cipherBytes))
|
||||
}
|
||||
@ -166,7 +166,7 @@ class KeyProviderFactoryTest {
|
||||
logger.info("Created temporary file based key provider: ${providerLocation}")
|
||||
|
||||
// Act
|
||||
KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], MASTER_KEY)
|
||||
KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], ROOT_KEY)
|
||||
logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
|
||||
|
||||
// Assert
|
||||
@ -184,7 +184,7 @@ class KeyProviderFactoryTest {
|
||||
logger.info("Created temporary file based key provider: ${providerLocation}")
|
||||
|
||||
// Act
|
||||
KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], MASTER_KEY)
|
||||
KeyProvider keyProvider = KeyProviderFactory.buildKeyProvider(fileBasedProvider, providerLocation, KEY_ID, [(KEY_ID): KEY_HEX], ROOT_KEY)
|
||||
logger.info("Key Provider ${fileBasedProvider} with location ${providerLocation} and keyId ${KEY_ID} / ${KEY_HEX} formed: ${keyProvider}")
|
||||
|
||||
// Assert
|
||||
@ -193,7 +193,7 @@ class KeyProviderFactoryTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testShouldNotBuildFileBasedKeyProviderWithoutMasterKey() {
|
||||
void testShouldNotBuildFileBasedKeyProviderWithoutRootKey() {
|
||||
// Arrange
|
||||
String fileBasedProvider = FileBasedKeyProvider.class.name
|
||||
File fileBasedProviderFile = tempFolder.newFile("filebased.kp")
|
||||
@ -208,7 +208,7 @@ class KeyProviderFactoryTest {
|
||||
}
|
||||
|
||||
// Assert
|
||||
assert msg =~ "The master key must be provided to decrypt the individual keys"
|
||||
assert msg =~ "The root key must be provided to decrypt the individual keys"
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -15,5 +15,5 @@
|
||||
# limitations under the License.
|
||||
#
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA9876543210
|
@ -15,5 +15,5 @@
|
||||
# limitations under the License.
|
||||
#
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
@ -16,6 +16,15 @@
|
||||
*/
|
||||
package org.apache.nifi.remote.protocol.socket;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.remote.Peer;
|
||||
import org.apache.nifi.remote.PeerDescription;
|
||||
@ -39,19 +48,9 @@ import org.apache.nifi.remote.protocol.ResponseCode;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class SocketClientProtocol implements ClientProtocol {
|
||||
|
||||
// Version 6 added to support Zero-Master Clustering, which was introduced in NiFi 1.0.0
|
||||
// Version 6 added to support Zero-Leader Clustering, which was introduced in NiFi 1.0.0
|
||||
private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(6, 5, 4, 3, 2, 1);
|
||||
|
||||
private RemoteDestination destination;
|
||||
@ -101,11 +100,11 @@ public class SocketClientProtocol implements ClientProtocol {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handshake(final Peer peer) throws IOException, HandshakeException {
|
||||
public void handshake(final Peer peer) throws IOException {
|
||||
handshake(peer, destination.getIdentifier());
|
||||
}
|
||||
|
||||
public void handshake(final Peer peer, final String destinationId) throws IOException, HandshakeException {
|
||||
public void handshake(final Peer peer, final String destinationId) throws IOException {
|
||||
if (handshakeComplete) {
|
||||
throw new IllegalStateException("Handshake has already been completed");
|
||||
}
|
||||
@ -173,8 +172,7 @@ public class SocketClientProtocol implements ClientProtocol {
|
||||
readyForFileTransfer = true;
|
||||
break;
|
||||
default:
|
||||
logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[]{
|
||||
this, handshakeResponse, peer});
|
||||
logger.error("{} received unexpected response {} from {} when negotiating Codec", this, handshakeResponse, peer);
|
||||
peer.close();
|
||||
throw new HandshakeException("Received unexpected response " + handshakeResponse);
|
||||
}
|
||||
@ -237,7 +235,7 @@ public class SocketClientProtocol implements ClientProtocol {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
|
||||
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException {
|
||||
if (!handshakeComplete) {
|
||||
throw new IllegalStateException("Handshake has not been performed");
|
||||
}
|
||||
@ -255,13 +253,13 @@ public class SocketClientProtocol implements ClientProtocol {
|
||||
} catch (HandshakeException e) {
|
||||
throw new ProtocolException(e.toString());
|
||||
}
|
||||
logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[]{this, codec, commsSession});
|
||||
logger.debug("{} negotiated FlowFileCodec {} with {}", this, codec, commsSession);
|
||||
|
||||
return codec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Transaction startTransaction(final Peer peer, final FlowFileCodec codec, final TransferDirection direction) throws IOException, ProtocolException {
|
||||
public Transaction startTransaction(final Peer peer, final FlowFileCodec codec, final TransferDirection direction) throws IOException {
|
||||
if (!handshakeComplete) {
|
||||
throw new IllegalStateException("Handshake has not been performed");
|
||||
}
|
||||
|
@ -1583,11 +1583,13 @@ For more information about each utility, see the link:toolkit-guide.html[NiFi To
|
||||
This section provides a quick overview of NiFi Clustering and instructions on how to set up a basic cluster.
|
||||
In the future, we hope to provide supplemental documentation that covers the NiFi Cluster Architecture in depth.
|
||||
|
||||
image::zero-master-cluster-http-access.png["NiFi Cluster HTTP Access"]
|
||||
image::zero-leader-cluster-http-access.png["NiFi Cluster HTTP Access"]
|
||||
|
||||
=== Zero-Master Clustering
|
||||
NiFi employs a Zero-Master Clustering paradigm. Each node in the cluster performs the same tasks on
|
||||
the data, but each operates on a different set of data. One of the nodes is automatically elected (via Apache
|
||||
=== Zero-Leader Clustering
|
||||
NiFi employs a Zero-Leader Clustering paradigm. Each node in the cluster has an identical flow definition and performs the same tasks on
|
||||
the data, but each operates on a different set of data. The cluster automatically distributes the data throughout all the active nodes.
|
||||
|
||||
One of the nodes is automatically elected (via Apache
|
||||
ZooKeeper) as the Cluster Coordinator. All nodes in the cluster will then send heartbeat/status information
|
||||
to this node, and this node is responsible for disconnecting nodes that do not report any heartbeat status
|
||||
for some amount of time. Additionally, when a new node elects to join the cluster, the new node must first
|
||||
@ -2173,9 +2175,9 @@ take effect only after NiFi has been stopped and restarted.
|
||||
|`java.arg.N`|Any number of JVM arguments can be passed to the NiFi JVM when the process is started. These arguments are defined by adding properties to _bootstrap.conf_ that
|
||||
begin with `java.arg.`. The rest of the property name is not relevant, other than to differentiate property names, and will be ignored. The default includes
|
||||
properties for minimum and maximum Java Heap size, the garbage collector to use, etc.
|
||||
|`nifi.bootstrap.sensitive.key`|The master key in hexadecimal format for encrypted sensitive configuration values. When NiFi is started, the master key is used to decrypt sensitive values from the _nifi.properties_ file into memory for later use.
|
||||
|`nifi.bootstrap.sensitive.key`|The root key (in hexadecimal format) for encrypted sensitive configuration values. When NiFi is started, this root key is used to decrypt sensitive values from the _nifi.properties_ file into memory for later use.
|
||||
|
||||
The Encrypt-Config Tool can be used to specify the master key, encrypt sensitive values in _nifi.properties_ and update _bootstrap.conf_. See the <<toolkit-guide.adoc#encrypt_config_tool,NiFi Toolkit Guide>> for an example.
|
||||
The Encrypt-Config Tool can be used to specify the root key, encrypt sensitive values in _nifi.properties_ and update _bootstrap.conf_. See the <<toolkit-guide.adoc#encrypt_config_tool,NiFi Toolkit Guide>> for an example.
|
||||
|`notification.services.file`|When NiFi is started, or stopped, or when the Bootstrap detects that NiFi has died, the Bootstrap is able to send notifications of these events
|
||||
to interested parties. This is configured by specifying an XML file that defines which notification services can be used. More about this
|
||||
file can be found in the <<notification_services>> section.
|
||||
|
@ -1995,7 +1995,7 @@ There are two UI extension points that are available in NiFi:
|
||||
- Custom Processor UIs
|
||||
- Content Viewers
|
||||
|
||||
Custom UIs can be created to provide configuration options beyond the standard property/value tables available in most processor settings. Examples of processors with Custom UIs are link:https://github.com/apache/nifi/tree/master/nifi-nar-bundles/nifi-update-attribute-bundle[UpdateAttribute^] and link:https://github.com/apache/nifi/tree/master/nifi-nar-bundles/nifi-standard-bundle[JoltTransformJSON^].
|
||||
Custom UIs can be created to provide configuration options beyond the standard property/value tables available in most processor settings. Examples of processors with Custom UIs are link:https://github.com/apache/nifi/tree/main/nifi-nar-bundles/nifi-update-attribute-bundle[UpdateAttribute^] and link:https://github.com/apache/nifi/tree/main/nifi-nar-bundles/nifi-standard-bundle[JoltTransformJSON^].
|
||||
|
||||
Content Viewers can be created to extend the types of data that can be viewed within NiFi. NiFi comes with NARs in the lib directory which contain content viewers for data types such as csv, xml, avro, json (standard-nar) and image types such as png, jpeg and gif (media-nar).
|
||||
|
||||
|
Before Width: | Height: | Size: 76 KiB After Width: | Height: | Size: 76 KiB |
Before Width: | Height: | Size: 66 KiB After Width: | Height: | Size: 66 KiB |
Before Width: | Height: | Size: 44 KiB After Width: | Height: | Size: 44 KiB |
@ -33,7 +33,7 @@ There are three repositories that are utilized by NiFi. Each exists within the O
|
||||
- The Content Repository holds the content for current and past FlowFiles.
|
||||
- The Provenance Repository holds the history of FlowFiles.
|
||||
|
||||
image::zero-master-node.png["NiFi Architecture Diagram"]
|
||||
image::zero-leader-node.png["NiFi Architecture Diagram"]
|
||||
|
||||
=== FlowFile Repository
|
||||
FlowFiles that are actively being processed by the system are held in a hash map in the JVM memory (more about that in <<DeeperView>>). This makes it very efficient to process them, but requires a secondary mechanism to provide durability of data across process restarts due to a number of reasons, such as power loss, kernel panics, system upgrades, and maintenance cycles. The FlowFile Repository is a "Write-Ahead Log" (or data record) of the metadata of each of the FlowFiles that currently exist in the system. This FlowFile metadata includes all the attributes associated with the FlowFile, a pointer to the actual content of the FlowFile (which exists in the Content Repo) and the state of the FlowFile, such as which Connection/Queue the FlowFile belongs in. This Write-Ahead Log provides NiFi the resiliency it needs to handle restarts and unexpected system failures.
|
||||
|
@ -120,7 +120,7 @@ A few of these benefits include:
|
||||
* The points at which data enters and exits the system as well as how it flows through are well understood and easily tracked
|
||||
|
||||
== NiFi Architecture
|
||||
image::zero-master-node.png["NiFi Architecture Diagram"]
|
||||
image::zero-leader-node.png["NiFi Architecture Diagram"]
|
||||
|
||||
NiFi executes within a JVM on a host operating system. The primary
|
||||
components of NiFi on the JVM are as follows:
|
||||
@ -145,9 +145,9 @@ The Provenance Repository is where all provenance event data is stored. The rep
|
||||
|
||||
NiFi is also able to operate within a cluster.
|
||||
|
||||
image::zero-master-cluster.png["NiFi Cluster Architecture Diagram"]
|
||||
image::zero-leader-cluster.png["NiFi Cluster Architecture Diagram"]
|
||||
|
||||
Starting with the NiFi 1.0 release, a Zero-Master Clustering paradigm is employed. Each node in a NiFi cluster performs the same tasks on the data, but each operates on a different set of data. Apache ZooKeeper elects a single node as the Cluster Coordinator, and failover is handled automatically by ZooKeeper. All cluster nodes report heartbeat and status information to the Cluster Coordinator. The Cluster Coordinator is responsible for disconnecting and connecting nodes. Additionally, every cluster has one Primary Node, also elected by ZooKeeper. As a DataFlow manager, you can interact with the NiFi cluster through the user interface (UI) of any node. Any change you make is replicated to all nodes in the cluster, allowing for multiple entry points.
|
||||
Starting with the NiFi 1.0 release, a Zero-Leader Clustering paradigm is employed. Each node in a NiFi cluster performs the same tasks on the data, but each operates on a different set of data. Apache ZooKeeper elects a single node as the Cluster Coordinator, and failover is handled automatically by ZooKeeper. All cluster nodes report heartbeat and status information to the Cluster Coordinator. The Cluster Coordinator is responsible for disconnecting and connecting nodes. Additionally, every cluster has one Primary Node, also elected by ZooKeeper. As a DataFlow manager, you can interact with the NiFi cluster through the user interface (UI) of any node. Any change you make is replicated to all nodes in the cluster, allowing for multiple entry points.
|
||||
|
||||
== Performance Expectations and Characteristics of NiFi
|
||||
NiFi is designed to fully leverage the capabilities of the underlying host system
|
||||
|
@ -360,7 +360,7 @@ To add a NiFi Registry command, perform the same steps, but extend from `Abstrac
|
||||
|
||||
[[encrypt_config_tool]]
|
||||
== Encrypt-Config Tool
|
||||
The `encrypt-config` command line tool (invoked as `./bin/encrypt-config.sh` or `bin\encrypt-config.bat`) reads from a _nifi.properties_ file with plaintext sensitive configuration values, prompts for a master password or raw hexadecimal key, and encrypts each value. It replaces the plain values with the protected value in the same file, or writes to a new _nifi.properties_ file if specified.
|
||||
The `encrypt-config` command line tool (invoked as `./bin/encrypt-config.sh` or `bin\encrypt-config.bat`) reads from a _nifi.properties_ file with plaintext sensitive configuration values, prompts for a root password or raw hexadecimal key, and encrypts each value. It replaces the plain values with the protected value in the same file, or writes to a new _nifi.properties_ file if specified.
|
||||
|
||||
The default encryption algorithm utilized is AES/GCM 128/256-bit. 128-bit is used if the JCE Unlimited Strength Cryptographic Jurisdiction Policy files are not installed, and 256-bit is used if they are installed.
|
||||
|
||||
@ -377,7 +377,7 @@ The following are available options:
|
||||
* `-l`,`--loginIdentityProviders <arg>` The _login-identity-providers.xml_ file containing unprotected config values (will be overwritten)
|
||||
* `-a`,`--authorizers <arg>` The _authorizers.xml_ file containing unprotected config values (will be overwritten)
|
||||
* `-f`,`--flowXml <arg>` The _flow.xml.gz_ file currently protected with old password (will be overwritten)
|
||||
* `-b`,`--bootstrapConf <arg>` The _bootstrap.conf_ file to persist master key
|
||||
* `-b`,`--bootstrapConf <arg>` The _bootstrap.conf_ file to persist root key
|
||||
* `-o`,`--outputNiFiProperties <arg>` The destination _nifi.properties_ file containing protected config values (will not modify input _nifi.properties_)
|
||||
* `-i`,`--outputLoginIdentityProviders <arg>` The destination _login-identity-providers.xml_ file containing protected config values (will not modify input _login-identity-providers.xml_)
|
||||
* `-u`,`--outputAuthorizers <arg>` The destination _authorizers.xml_ file containing protected config values (will not modify input _authorizers.xml_)
|
||||
@ -447,7 +447,7 @@ Additionally, the _bootstrap.conf_ file is updated with the encryption key as fo
|
||||
|
||||
[source]
|
||||
----
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
||||
----
|
||||
|
||||
|
@ -2838,7 +2838,7 @@ key4=kZprfcTSTH69UuOU3jMkZfrtiVR/eqWmmbdku3bQcUJ/+UToecNB5lzOVEMBChyEXppyXXC35Wa
|
||||
key5=c6FzfnKm7UR7xqI2NFpZ+fEKBfSU7+1NvRw+XWQ9U39MONWqk5gvoyOCdFR1kUgeg46jrN5dGXk13sRqE0GETQ==
|
||||
....
|
||||
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **master key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **root key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
|
||||
[[provenance-repository-key-rotation]]
|
||||
===== Key Rotation
|
||||
@ -2917,7 +2917,7 @@ key4=kZprfcTSTH69UuOU3jMkZfrtiVR/eqWmmbdku3bQcUJ/+UToecNB5lzOVEMBChyEXppyXXC35Wa
|
||||
key5=c6FzfnKm7UR7xqI2NFpZ+fEKBfSU7+1NvRw+XWQ9U39MONWqk5gvoyOCdFR1kUgeg46jrN5dGXk13sRqE0GETQ==
|
||||
....
|
||||
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **master key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **root key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
|
||||
.Data Protection vs. Key Protection
|
||||
****
|
||||
@ -3003,7 +3003,7 @@ key4=kZprfcTSTH69UuOU3jMkZfrtiVR/eqWmmbdku3bQcUJ/+UToecNB5lzOVEMBChyEXppyXXC35Wa
|
||||
key5=c6FzfnKm7UR7xqI2NFpZ+fEKBfSU7+1NvRw+XWQ9U39MONWqk5gvoyOCdFR1kUgeg46jrN5dGXk13sRqE0GETQ==
|
||||
....
|
||||
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **master key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
Each line defines a key ID and then the Base64-encoded cipher text of a 16 byte IV and wrapped AES-128, AES-192, or AES-256 key depending on the JCE policies available. The individual keys are wrapped by AES/GCM encryption using the **root key** defined by `nifi.bootstrap.sensitive.key` in _conf/bootstrap.conf_.
|
||||
|
||||
[[flowfile-repository-key-rotation]]
|
||||
==== Key Rotation
|
||||
|
@ -16,6 +16,15 @@
|
||||
*/
|
||||
package org.apache.nifi.cdc.mysql.processors;
|
||||
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.EXT_DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.EXT_WRITE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.FORMAT_DESCRIPTION;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_WRITE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.ROTATE;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.WRITE_ROWS;
|
||||
|
||||
import com.github.shyiko.mysql.binlog.BinaryLogClient;
|
||||
import com.github.shyiko.mysql.binlog.event.Event;
|
||||
import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
|
||||
@ -23,6 +32,34 @@ import com.github.shyiko.mysql.binlog.event.EventType;
|
||||
import com.github.shyiko.mysql.binlog.event.QueryEventData;
|
||||
import com.github.shyiko.mysql.binlog.event.RotateEventData;
|
||||
import com.github.shyiko.mysql.binlog.event.TableMapEventData;
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.MalformedURLException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.Driver;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.DriverPropertyInfo;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.SQLFeatureNotSupportedException;
|
||||
import java.sql.Statement;
|
||||
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.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.logging.Logger;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.Stateful;
|
||||
@ -44,16 +81,16 @@ import org.apache.nifi.cdc.mysql.event.BinlogEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.BinlogEventListener;
|
||||
import org.apache.nifi.cdc.mysql.event.BinlogLifecycleListener;
|
||||
import org.apache.nifi.cdc.mysql.event.CommitTransactionEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.DDLEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.DeleteRowsEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.InsertRowsEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.RawBinlogEvent;
|
||||
import org.apache.nifi.cdc.mysql.event.DDLEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.UpdateRowsEventInfo;
|
||||
import org.apache.nifi.cdc.mysql.event.io.BeginTransactionEventWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.CommitTransactionEventWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.DDLEventWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.DeleteRowsWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.InsertRowsWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.DDLEventWriter;
|
||||
import org.apache.nifi.cdc.mysql.event.io.UpdateRowsWriter;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
@ -75,45 +112,6 @@ import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.MalformedURLException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.Driver;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.DriverPropertyInfo;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.SQLFeatureNotSupportedException;
|
||||
import java.sql.Statement;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
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.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.logging.Logger;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.EXT_DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.EXT_WRITE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.FORMAT_DESCRIPTION;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_DELETE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.PRE_GA_WRITE_ROWS;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.ROTATE;
|
||||
import static com.github.shyiko.mysql.binlog.event.EventType.WRITE_ROWS;
|
||||
|
||||
|
||||
/**
|
||||
* A processor to retrieve Change Data Capture (CDC) events and send them as flow files.
|
||||
@ -233,8 +231,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
public static final PropertyDescriptor SERVER_ID = new PropertyDescriptor.Builder()
|
||||
.name("capture-change-mysql-server-id")
|
||||
.displayName("Server ID")
|
||||
.description("The client connecting to the MySQL replication group is actually a simplified slave (server), and the Server ID value must be unique across the whole replication "
|
||||
+ "group (i.e. different from any other Server ID being used by any master or slave). Thus, each instance of CaptureChangeMySQL must have a Server ID unique across "
|
||||
.description("The client connecting to the MySQL replication group is actually a simplified replica (server), and the Server ID value must be unique across the whole replication "
|
||||
+ "group (i.e. different from any other Server ID being used by any primary or replica). Thus, each instance of CaptureChangeMySQL must have a Server ID unique across "
|
||||
+ "the replication group. If the Server ID is not specified, it defaults to 65535.")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
|
||||
@ -337,14 +335,14 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
|
||||
.build();
|
||||
|
||||
private static List<PropertyDescriptor> propDescriptors;
|
||||
private static final List<PropertyDescriptor> propDescriptors;
|
||||
|
||||
private volatile ProcessSession currentSession;
|
||||
private BinaryLogClient binlogClient;
|
||||
private BinlogEventListener eventListener;
|
||||
private BinlogLifecycleListener lifecycleListener;
|
||||
|
||||
private volatile LinkedBlockingQueue<RawBinlogEvent> queue = new LinkedBlockingQueue<>();
|
||||
private final LinkedBlockingQueue<RawBinlogEvent> queue = new LinkedBlockingQueue<>();
|
||||
private volatile String currentBinlogFile = null;
|
||||
private volatile long currentBinlogPosition = 4;
|
||||
|
||||
@ -362,15 +360,15 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
|
||||
private volatile boolean inTransaction = false;
|
||||
private volatile boolean skipTable = false;
|
||||
private AtomicBoolean doStop = new AtomicBoolean(false);
|
||||
private AtomicBoolean hasRun = new AtomicBoolean(false);
|
||||
private final AtomicBoolean doStop = new AtomicBoolean(false);
|
||||
private final AtomicBoolean hasRun = new AtomicBoolean(false);
|
||||
|
||||
private int currentHost = 0;
|
||||
private String transitUri = "<unknown>";
|
||||
|
||||
private volatile long lastStateUpdate = 0L;
|
||||
private volatile long stateUpdateInterval = -1L;
|
||||
private AtomicLong currentSequenceId = new AtomicLong(0);
|
||||
private final AtomicLong currentSequenceId = new AtomicLong(0);
|
||||
|
||||
private volatile DistributedMapCacheClient cacheClient = null;
|
||||
private final Serializer<TableInfoCacheKey> cacheKeySerializer = new TableInfoCacheKey.Serializer();
|
||||
@ -624,7 +622,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
if (hostsString == null) {
|
||||
return null;
|
||||
}
|
||||
final List<String> hostsSplit = Arrays.asList(hostsString.split(","));
|
||||
final String[] hostsSplit = hostsString.split(",");
|
||||
List<InetSocketAddress> hostsList = new ArrayList<>();
|
||||
|
||||
for (String item : hostsSplit) {
|
||||
@ -1030,9 +1028,9 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
}
|
||||
|
||||
private class JDBCConnectionHolder {
|
||||
private String connectionUrl;
|
||||
private Properties connectionProps = new Properties();
|
||||
private long connectionTimeoutMillis;
|
||||
private final String connectionUrl;
|
||||
private final Properties connectionProps = new Properties();
|
||||
private final long connectionTimeoutMillis;
|
||||
|
||||
private Connection connection;
|
||||
|
||||
@ -1108,7 +1106,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
|
||||
}
|
||||
|
||||
private static class DriverShim implements Driver {
|
||||
private Driver driver;
|
||||
private final Driver driver;
|
||||
|
||||
DriverShim(Driver d) {
|
||||
this.driver = d;
|
||||
|
@ -16,39 +16,6 @@
|
||||
*/
|
||||
package org.apache.nifi.authorization;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.authorization.annotation.AuthorizerContext;
|
||||
import org.apache.nifi.authorization.exception.AuthorizationAccessException;
|
||||
import org.apache.nifi.authorization.exception.AuthorizerCreationException;
|
||||
import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
|
||||
import org.apache.nifi.authorization.generated.Authorizers;
|
||||
import org.apache.nifi.authorization.generated.Property;
|
||||
import org.apache.nifi.bundle.Bundle;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.properties.AESSensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.properties.NiFiPropertiesLoader;
|
||||
import org.apache.nifi.properties.SensitivePropertyProtectionException;
|
||||
import org.apache.nifi.properties.SensitivePropertyProvider;
|
||||
import org.apache.nifi.properties.SensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.security.xml.XmlUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.springframework.beans.factory.DisposableBean;
|
||||
import org.springframework.beans.factory.FactoryBean;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
import javax.xml.XMLConstants;
|
||||
import javax.xml.bind.JAXBContext;
|
||||
import javax.xml.bind.JAXBElement;
|
||||
import javax.xml.bind.JAXBException;
|
||||
import javax.xml.bind.Unmarshaller;
|
||||
import javax.xml.stream.XMLStreamException;
|
||||
import javax.xml.stream.XMLStreamReader;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
import javax.xml.validation.Schema;
|
||||
import javax.xml.validation.SchemaFactory;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
@ -62,6 +29,38 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.xml.XMLConstants;
|
||||
import javax.xml.bind.JAXBContext;
|
||||
import javax.xml.bind.JAXBElement;
|
||||
import javax.xml.bind.JAXBException;
|
||||
import javax.xml.bind.Unmarshaller;
|
||||
import javax.xml.stream.XMLStreamException;
|
||||
import javax.xml.stream.XMLStreamReader;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
import javax.xml.validation.Schema;
|
||||
import javax.xml.validation.SchemaFactory;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.authorization.annotation.AuthorizerContext;
|
||||
import org.apache.nifi.authorization.exception.AuthorizationAccessException;
|
||||
import org.apache.nifi.authorization.exception.AuthorizerCreationException;
|
||||
import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
|
||||
import org.apache.nifi.authorization.generated.Authorizers;
|
||||
import org.apache.nifi.authorization.generated.Property;
|
||||
import org.apache.nifi.bundle.Bundle;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.properties.AESSensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.properties.SensitivePropertyProtectionException;
|
||||
import org.apache.nifi.properties.SensitivePropertyProvider;
|
||||
import org.apache.nifi.properties.SensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.security.kms.CryptoUtils;
|
||||
import org.apache.nifi.security.xml.XmlUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.springframework.beans.factory.DisposableBean;
|
||||
import org.springframework.beans.factory.FactoryBean;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
/**
|
||||
* Factory bean for loading the configured authorizer.
|
||||
@ -489,18 +488,18 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
|
||||
private static void initializeSensitivePropertyProvider(String encryptionScheme) throws SensitivePropertyProtectionException {
|
||||
if (SENSITIVE_PROPERTY_PROVIDER == null || !SENSITIVE_PROPERTY_PROVIDER.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
|
||||
try {
|
||||
String keyHex = getMasterKey();
|
||||
String keyHex = getRootKey();
|
||||
SENSITIVE_PROPERTY_PROVIDER_FACTORY = new AESSensitivePropertyProviderFactory(keyHex);
|
||||
SENSITIVE_PROPERTY_PROVIDER = SENSITIVE_PROPERTY_PROVIDER_FACTORY.getProvider();
|
||||
} catch (IOException e) {
|
||||
logger.error("Error extracting master key from bootstrap.conf for login identity provider decryption", e);
|
||||
throw new SensitivePropertyProtectionException("Could not read master key from bootstrap.conf");
|
||||
logger.error("Error extracting root key from bootstrap.conf for login identity provider decryption", e);
|
||||
throw new SensitivePropertyProtectionException("Could not read root key from bootstrap.conf");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static String getMasterKey() throws IOException {
|
||||
return NiFiPropertiesLoader.extractKeyFromBootstrapFile();
|
||||
private static String getRootKey() throws IOException {
|
||||
return CryptoUtils.extractKeyFromBootstrapFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -87,7 +87,7 @@ public class NiFiPropertiesLoader {
|
||||
*/
|
||||
public static NiFiProperties loadDefaultWithKeyFromBootstrap() throws IOException {
|
||||
try {
|
||||
String keyHex = extractKeyFromBootstrapFile();
|
||||
String keyHex = CryptoUtils.extractKeyFromBootstrapFile();
|
||||
return NiFiPropertiesLoader.withKey(keyHex).loadDefault();
|
||||
} catch (IOException e) {
|
||||
logger.error("Encountered an exception loading the default nifi.properties file {} with the key provided in bootstrap.conf", CryptoUtils.getDefaultFilePath(), e);
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
||||
|
||||
###
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=
|
||||
|
||||
###
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
||||
|
||||
###
|
||||
|
@ -52,7 +52,7 @@ java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=
|
||||
|
||||
# Sets the provider of SecureRandom to /dev/urandom to prevent blocking on VMs
|
||||
|
@ -48,7 +48,7 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
|
||||
public static final String RESOURCE_NAME = "SocketFlowFileProtocol";
|
||||
|
||||
// Version 6 added to support Zero-Master Clustering, which was introduced in NiFi 1.0.0
|
||||
// Version 6 added to support Zero-Leader Clustering, which was introduced in NiFi 1.0.0
|
||||
private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(6, 5, 4, 3, 2, 1);
|
||||
|
||||
private PeerDescriptionModifier peerDescriptionModifier;
|
||||
@ -59,7 +59,7 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HandshakeProperties doHandshake(Peer peer) throws IOException, HandshakeException {
|
||||
protected HandshakeProperties doHandshake(Peer peer) throws IOException {
|
||||
|
||||
HandshakeProperties confirmed = new HandshakeProperties();
|
||||
|
||||
@ -117,7 +117,7 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
|
||||
public FlowFileCodec negotiateCodec(final Peer peer) throws IOException {
|
||||
if (!handshakeCompleted) {
|
||||
throw new IllegalStateException("Handshake has not been completed");
|
||||
}
|
||||
@ -125,7 +125,7 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
throw new IllegalStateException("Protocol is shutdown");
|
||||
}
|
||||
|
||||
logger.debug("{} Negotiating Codec with {} using {}", new Object[]{this, peer, peer.getCommunicationsSession()});
|
||||
logger.debug("{} Negotiating Codec with {} using {}", this, peer, peer.getCommunicationsSession());
|
||||
final CommunicationsSession commsSession = peer.getCommunicationsSession();
|
||||
final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
|
||||
final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
|
||||
@ -137,7 +137,7 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
// Negotiate the FlowFileCodec to use.
|
||||
try {
|
||||
negotiatedFlowFileCodec = RemoteResourceFactory.receiveCodecNegotiation(dis, dos);
|
||||
logger.debug("{} Negotiated Codec {} with {}", new Object[]{this, negotiatedFlowFileCodec, peer});
|
||||
logger.debug("{} Negotiated Codec {} with {}", this, negotiatedFlowFileCodec, peer);
|
||||
return negotiatedFlowFileCodec;
|
||||
} catch (final HandshakeException e) {
|
||||
throw new ProtocolException(e.toString());
|
||||
@ -154,9 +154,9 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
|
||||
throw new IllegalStateException("Protocol is shutdown");
|
||||
}
|
||||
|
||||
logger.debug("{} Reading Request Type from {} using {}", new Object[]{this, peer, peer.getCommunicationsSession()});
|
||||
logger.debug("{} Reading Request Type from {} using {}", this, peer, peer.getCommunicationsSession());
|
||||
final RequestType requestType = RequestType.readRequestType(new DataInputStream(peer.getCommunicationsSession().getInput().getInputStream()));
|
||||
logger.debug("{} Got Request Type {} from {}", new Object[]{this, requestType, peer});
|
||||
logger.debug("{} Got Request Type {} from {}", this, requestType, peer);
|
||||
|
||||
return requestType;
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
||||
|
||||
###
|
||||
|
@ -51,10 +51,10 @@ import org.apache.nifi.bundle.Bundle;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.nar.NarCloseable;
|
||||
import org.apache.nifi.properties.AESSensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.properties.NiFiPropertiesLoader;
|
||||
import org.apache.nifi.properties.SensitivePropertyProtectionException;
|
||||
import org.apache.nifi.properties.SensitivePropertyProvider;
|
||||
import org.apache.nifi.properties.SensitivePropertyProviderFactory;
|
||||
import org.apache.nifi.security.kms.CryptoUtils;
|
||||
import org.apache.nifi.security.xml.XmlUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.slf4j.Logger;
|
||||
@ -226,18 +226,18 @@ public class LoginIdentityProviderFactoryBean implements FactoryBean, Disposable
|
||||
private static void initializeSensitivePropertyProvider(String encryptionScheme) throws SensitivePropertyProtectionException {
|
||||
if (SENSITIVE_PROPERTY_PROVIDER == null || !SENSITIVE_PROPERTY_PROVIDER.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
|
||||
try {
|
||||
String keyHex = getMasterKey();
|
||||
String keyHex = getRootKey();
|
||||
SENSITIVE_PROPERTY_PROVIDER_FACTORY = new AESSensitivePropertyProviderFactory(keyHex);
|
||||
SENSITIVE_PROPERTY_PROVIDER = SENSITIVE_PROPERTY_PROVIDER_FACTORY.getProvider();
|
||||
} catch (IOException e) {
|
||||
logger.error("Error extracting master key from bootstrap.conf for login identity provider decryption", e);
|
||||
throw new SensitivePropertyProtectionException("Could not read master key from bootstrap.conf");
|
||||
throw new SensitivePropertyProtectionException("Could not read root key from bootstrap.conf");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static String getMasterKey() throws IOException {
|
||||
return NiFiPropertiesLoader.extractKeyFromBootstrapFile();
|
||||
private static String getRootKey() throws IOException {
|
||||
return CryptoUtils.extractKeyFromBootstrapFile();
|
||||
}
|
||||
|
||||
private void performMethodInjection(final LoginIdentityProvider instance, final Class loginIdentityProviderClass)
|
||||
|
@ -54,7 +54,7 @@
|
||||
</tr>
|
||||
<tr>
|
||||
<td>CTL</td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/master/nifi-api/src/main/java/org/apache/nifi/controller/ControllerService.java">ControllerService</a>></td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/main/nifi-api/src/main/java/org/apache/nifi/controller/ControllerService.java">ControllerService</a>></td>
|
||||
<td>Map populated with controller services defined with `CTL.*` processor properties.
|
||||
<br/>The `CTL.` prefixed properties could be linked to controller service and provides access to this service from a script without additional code.</td>
|
||||
</tr>
|
||||
@ -66,13 +66,13 @@
|
||||
</tr>
|
||||
<tr>
|
||||
<td>RecordReader</td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java">RecordReaderFactory</a>></td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java">RecordReaderFactory</a>></td>
|
||||
<td>Map populated with controller services defined with `RecordReader.*` processor properties.
|
||||
<br/>The `RecordReader.` prefixed properties are to be linked to RecordReaderFactory controller service instances.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>RecordWriter</td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java">RecordSetWriterFactory</a>></td>
|
||||
<td>java.util.HashMap<String,<a href="https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java">RecordSetWriterFactory</a>></td>
|
||||
<td>Map populated with controller services defined with `RecordWriter.*` processor properties.
|
||||
<br/>The `RecordWriter.` prefixed properties are to be linked to RecordSetWriterFactory controller service instances.</td>
|
||||
</tr>
|
||||
|
@ -24,7 +24,6 @@ import org.apache.commons.codec.DecoderException;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.properties.NiFiPropertiesLoader;
|
||||
import org.apache.nifi.provenance.serialization.RecordReaders;
|
||||
import org.apache.nifi.provenance.store.EventFileManager;
|
||||
import org.apache.nifi.provenance.store.RecordReaderFactory;
|
||||
@ -32,6 +31,7 @@ import org.apache.nifi.provenance.store.RecordWriterFactory;
|
||||
import org.apache.nifi.provenance.toc.StandardTocWriter;
|
||||
import org.apache.nifi.provenance.toc.TocUtil;
|
||||
import org.apache.nifi.provenance.toc.TocWriter;
|
||||
import org.apache.nifi.security.kms.CryptoUtils;
|
||||
import org.apache.nifi.security.kms.KeyProvider;
|
||||
import org.apache.nifi.security.kms.KeyProviderFactory;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
@ -84,9 +84,9 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
|
||||
if (getConfig().supportsEncryption()) {
|
||||
try {
|
||||
KeyProvider keyProvider;
|
||||
if (KeyProviderFactory.requiresMasterKey(getConfig().getKeyProviderImplementation())) {
|
||||
SecretKey masterKey = getMasterKey();
|
||||
keyProvider = buildKeyProvider(masterKey);
|
||||
if (KeyProviderFactory.requiresRootKey(getConfig().getKeyProviderImplementation())) {
|
||||
SecretKey rootKey = getRootKey();
|
||||
keyProvider = buildKeyProvider(rootKey);
|
||||
} else {
|
||||
keyProvider = buildKeyProvider();
|
||||
}
|
||||
@ -133,7 +133,7 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
|
||||
return buildKeyProvider(null);
|
||||
}
|
||||
|
||||
private KeyProvider buildKeyProvider(SecretKey masterKey) throws KeyManagementException {
|
||||
private KeyProvider buildKeyProvider(SecretKey rootKey) throws KeyManagementException {
|
||||
RepositoryConfiguration config = super.getConfig();
|
||||
if (config == null) {
|
||||
throw new KeyManagementException("The repository configuration is missing");
|
||||
@ -145,14 +145,14 @@ public class EncryptedWriteAheadProvenanceRepository extends WriteAheadProvenanc
|
||||
+ NiFiProperties.PROVENANCE_REPO_ENCRYPTION_KEY_PROVIDER_IMPLEMENTATION_CLASS);
|
||||
}
|
||||
|
||||
return KeyProviderFactory.buildKeyProvider(implementationClassName, config.getKeyProviderLocation(), config.getKeyId(), config.getEncryptionKeys(), masterKey);
|
||||
return KeyProviderFactory.buildKeyProvider(implementationClassName, config.getKeyProviderLocation(), config.getKeyId(), config.getEncryptionKeys(), rootKey);
|
||||
}
|
||||
|
||||
private static SecretKey getMasterKey() throws KeyManagementException {
|
||||
private static SecretKey getRootKey() throws KeyManagementException {
|
||||
try {
|
||||
// Get the master encryption key from bootstrap.conf
|
||||
String masterKeyHex = NiFiPropertiesLoader.extractKeyFromBootstrapFile();
|
||||
return new SecretKeySpec(Hex.decodeHex(masterKeyHex.toCharArray()), "AES");
|
||||
// Get the root encryption key from bootstrap.conf
|
||||
String rootKeyHex = CryptoUtils.extractKeyFromBootstrapFile();
|
||||
return new SecretKeySpec(Hex.decodeHex(rootKeyHex.toCharArray()), "AES");
|
||||
} catch (IOException | DecoderException e) {
|
||||
logger.error("Encountered an error: ", e);
|
||||
throw new KeyManagementException(e);
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=
|
||||
|
||||
###
|
||||
|
@ -198,6 +198,6 @@ nifi.variable.registry.properties=
|
||||
|
||||
# Build info
|
||||
nifi.build.tag=HEAD
|
||||
nifi.build.branch=master
|
||||
nifi.build.branch=main
|
||||
nifi.build.revision=868795c
|
||||
nifi.build.timestamp=2016-12-13T15:41:36Z
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=
|
||||
|
||||
###
|
||||
|
@ -28,6 +28,7 @@ import org.apache.commons.cli.Options
|
||||
import org.apache.commons.cli.ParseException
|
||||
import org.apache.commons.codec.binary.Hex
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.nifi.security.kms.CryptoUtils
|
||||
import org.apache.nifi.toolkit.tls.commandLine.CommandLineParseException
|
||||
import org.apache.nifi.toolkit.tls.commandLine.ExitCode
|
||||
import org.apache.nifi.util.NiFiProperties
|
||||
@ -135,7 +136,7 @@ class ConfigEncryptionTool {
|
||||
private static final int DEFAULT_SALT_SIZE_BYTES = 16
|
||||
|
||||
private static
|
||||
final String BOOTSTRAP_KEY_COMMENT = "# Master key in hexadecimal format for encrypted sensitive configuration values"
|
||||
final String BOOTSTRAP_KEY_COMMENT = "# Root key in hexadecimal format for encrypted sensitive configuration values"
|
||||
private static final String BOOTSTRAP_KEY_PREFIX = "nifi.bootstrap.sensitive.key="
|
||||
private static final String JAVA_HOME = "JAVA_HOME"
|
||||
private static final String NIFI_TOOLKIT_HOME = "NIFI_TOOLKIT_HOME"
|
||||
@ -146,7 +147,7 @@ class ConfigEncryptionTool {
|
||||
private static
|
||||
final String DEFAULT_DESCRIPTION = "This tool reads from a nifi.properties and/or " +
|
||||
"login-identity-providers.xml file with plain sensitive configuration values, " +
|
||||
"prompts the user for a master key, and encrypts each value. It will replace the " +
|
||||
"prompts the user for a root key, and encrypts each value. It will replace the " +
|
||||
"plain value with the protected value in the same file (or write to a new file if " +
|
||||
"specified). It can also be used to migrate already-encrypted values in those " +
|
||||
"files or in flow.xml.gz to be encrypted with a new key."
|
||||
@ -232,7 +233,7 @@ class ConfigEncryptionTool {
|
||||
options.addOption(Option.builder("u").longOpt(OUTPUT_AUTHORIZERS_ARG).hasArg(true).argName("file").desc("The destination authorizers.xml file containing protected config values (will not modify input authorizers.xml)").build())
|
||||
options.addOption(Option.builder("f").longOpt(FLOW_XML_ARG).hasArg(true).argName("file").desc("The flow.xml.gz file currently protected with old password (will be overwritten unless -g is specified)").build())
|
||||
options.addOption(Option.builder("g").longOpt(OUTPUT_FLOW_XML_ARG).hasArg(true).argName("file").desc("The destination flow.xml.gz file containing protected config values (will not modify input flow.xml.gz)").build())
|
||||
options.addOption(Option.builder("b").longOpt(BOOTSTRAP_CONF_ARG).hasArg(true).argName("file").desc("The bootstrap.conf file to persist master key").build())
|
||||
options.addOption(Option.builder("b").longOpt(BOOTSTRAP_CONF_ARG).hasArg(true).argName("file").desc("The bootstrap.conf file to persist root key").build())
|
||||
options.addOption(Option.builder("k").longOpt(KEY_ARG).hasArg(true).argName("keyhex").desc("The raw hexadecimal key to use to encrypt the sensitive properties").build())
|
||||
options.addOption(Option.builder("e").longOpt(KEY_MIGRATION_ARG).hasArg(true).argName("keyhex").desc("The old raw hexadecimal key to use during key migration").build())
|
||||
options.addOption(Option.builder("p").longOpt(PASSWORD_ARG).hasArg(true).argName("password").desc("The password from which to derive the key to use to encrypt the sensitive properties").build())
|
||||
@ -538,7 +539,7 @@ class ConfigEncryptionTool {
|
||||
}
|
||||
|
||||
private static String readKeyFromConsole(TextDevice textDevice) {
|
||||
textDevice.printf("Enter the master key in hexadecimal format (spaces acceptable): ")
|
||||
textDevice.printf("Enter the root key in hexadecimal format (spaces acceptable): ")
|
||||
new String(textDevice.readPassword())
|
||||
}
|
||||
|
||||
@ -1039,7 +1040,7 @@ class ConfigEncryptionTool {
|
||||
}
|
||||
|
||||
/**
|
||||
* Accepts a {@link NiFiProperties} instance, iterates over all non-empty sensitive properties which are not already marked as protected, encrypts them using the master key, and updates the property with the protected value. Additionally, adds a new sibling property {@code x.y.z.protected=aes/gcm/{128,256}} for each indicating the encryption scheme used.
|
||||
* Accepts a {@link NiFiProperties} instance, iterates over all non-empty sensitive properties which are not already marked as protected, encrypts them using the root key, and updates the property with the protected value. Additionally, adds a new sibling property {@code x.y.z.protected=aes/gcm/{128,256}} for each indicating the encryption scheme used.
|
||||
*
|
||||
* @param plainProperties the NiFiProperties instance containing the raw values
|
||||
* @return the NiFiProperties containing protected values
|
||||
@ -1108,7 +1109,7 @@ class ConfigEncryptionTool {
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the existing {@code bootstrap.conf} file, updates it to contain the master key, and persists it back to the same location.
|
||||
* Reads the existing {@code bootstrap.conf} file, updates it to contain the root key, and persists it back to the same location.
|
||||
*
|
||||
* @throw IOException if there is a problem reading or writing the bootstrap.conf file
|
||||
*/
|
||||
@ -1123,7 +1124,7 @@ class ConfigEncryptionTool {
|
||||
// Write the updated values back to the file
|
||||
bootstrapConfFile.text = lines.join("\n")
|
||||
} catch (IOException e) {
|
||||
def msg = "Encountered an exception updating the bootstrap.conf file with the master key"
|
||||
def msg = "Encountered an exception updating the bootstrap.conf file with the root key"
|
||||
logger.error(msg, e)
|
||||
throw e
|
||||
}
|
||||
@ -1446,12 +1447,12 @@ class ConfigEncryptionTool {
|
||||
// Handle the translate CLI case
|
||||
if (tool.translatingCli) {
|
||||
if (tool.bootstrapConfPath) {
|
||||
// Check to see if bootstrap.conf has a master key
|
||||
tool.keyHex = NiFiPropertiesLoader.extractKeyFromBootstrapFile(tool.bootstrapConfPath)
|
||||
// Check to see if bootstrap.conf has a root key
|
||||
tool.keyHex = CryptoUtils.extractKeyFromBootstrapFile(tool.bootstrapConfPath)
|
||||
}
|
||||
|
||||
if (!tool.keyHex) {
|
||||
logger.info("No master key detected in ${tool.bootstrapConfPath} -- if ${tool.niFiPropertiesPath} is encrypted, the translation will fail")
|
||||
logger.info("No root key detected in ${tool.bootstrapConfPath} -- if ${tool.niFiPropertiesPath} is encrypted, the translation will fail")
|
||||
}
|
||||
|
||||
// Load the existing properties (decrypting if necessary)
|
||||
@ -1467,7 +1468,7 @@ class ConfigEncryptionTool {
|
||||
if (!tool.ignorePropertiesFiles || (tool.handlingFlowXml && existingNiFiPropertiesAreEncrypted)) {
|
||||
// If we are handling the flow.xml.gz and nifi.properties is already encrypted, try getting the key from bootstrap.conf rather than the console
|
||||
if (tool.ignorePropertiesFiles) {
|
||||
tool.keyHex = NiFiPropertiesLoader.extractKeyFromBootstrapFile(tool.bootstrapConfPath)
|
||||
tool.keyHex = CryptoUtils.extractKeyFromBootstrapFile(tool.bootstrapConfPath)
|
||||
} else {
|
||||
tool.keyHex = tool.getKey()
|
||||
}
|
||||
@ -1583,7 +1584,7 @@ class ConfigEncryptionTool {
|
||||
if (tool.isVerbose) {
|
||||
logger.error("Encountered an error", e)
|
||||
}
|
||||
tool.printUsageAndThrow("Encountered an error writing the master key to the bootstrap.conf file and the encrypted properties to nifi.properties", ExitCode.ERROR_GENERATING_CONFIG)
|
||||
tool.printUsageAndThrow("Encountered an error writing the root key to the bootstrap.conf file and the encrypted properties to nifi.properties", ExitCode.ERROR_GENERATING_CONFIG)
|
||||
}
|
||||
} catch (CommandLineParseException e) {
|
||||
System.exit(e.exitCode.ordinal())
|
||||
|
@ -192,7 +192,7 @@ class DecryptMode implements ToolMode {
|
||||
cli.b(longOpt: 'bootstrapConf',
|
||||
args: 1,
|
||||
argName: 'file',
|
||||
'Use a bootstrap.conf file containing the master key to decrypt the input file (as an alternative to -p or -k)')
|
||||
'Use a bootstrap.conf file containing the root key to decrypt the input file (as an alternative to -p or -k)')
|
||||
|
||||
cli.o(longOpt: 'output',
|
||||
args: 1,
|
||||
@ -249,7 +249,7 @@ class DecryptMode implements ToolMode {
|
||||
String validationFailedMessage = null
|
||||
|
||||
if (!rawOptions.b && !rawOptions.p && !rawOptions.k) {
|
||||
validationFailedMessage = "-p, -k, or -b is required in order to determine the master key to use for decryption."
|
||||
validationFailedMessage = "-p, -k, or -b is required in order to determine the root key to use for decryption."
|
||||
}
|
||||
|
||||
if (validationFailedMessage) {
|
||||
@ -292,18 +292,18 @@ class DecryptMode implements ToolMode {
|
||||
String password = null
|
||||
String keyHex = null
|
||||
if (usingPassword) {
|
||||
logger.debug("Using password to derive master key for decryption")
|
||||
logger.debug("Using password to derive root key for decryption")
|
||||
password = rawOptions.getOptionValue("p")
|
||||
keySource = Configuration.KeySource.PASSWORD
|
||||
} else {
|
||||
logger.debug("Using raw key hex as master key for decryption")
|
||||
logger.debug("Using raw key hex as root key for decryption")
|
||||
keyHex = rawOptions.getOptionValue("k")
|
||||
keySource = Configuration.KeySource.KEY_HEX
|
||||
}
|
||||
key = ToolUtilities.determineKey(TextDevices.defaultTextDevice(), keyHex, password, usingPassword)
|
||||
} else if (usingBootstrapKey) {
|
||||
inputBootstrapPath = rawOptions.b
|
||||
logger.debug("Looking in bootstrap conf file ${inputBootstrapPath} for master key for decryption.")
|
||||
logger.debug("Looking in bootstrap conf file ${inputBootstrapPath} for root key for decryption.")
|
||||
|
||||
// first, try to treat the bootstrap file as a NiFi bootstrap.conf
|
||||
logger.debug("Checking expected NiFi bootstrap.conf format")
|
||||
@ -317,10 +317,10 @@ class DecryptMode implements ToolMode {
|
||||
|
||||
// check we have found the key after trying all bootstrap formats
|
||||
if (key) {
|
||||
logger.debug("Master key found in ${inputBootstrapPath}. This key will be used for decryption operations.")
|
||||
logger.debug("Root key found in ${inputBootstrapPath}. This key will be used for decryption operations.")
|
||||
keySource = Configuration.KeySource.BOOTSTRAP_FILE
|
||||
} else {
|
||||
logger.warn("Bootstrap Conf flag present, but master key could not be found in ${inputBootstrapPath}.")
|
||||
logger.warn("Bootstrap Conf flag present, but root key could not be found in ${inputBootstrapPath}.")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -104,9 +104,9 @@ class NiFiRegistryDecryptMode extends DecryptMode {
|
||||
|
||||
// check we have found the key
|
||||
if (config.key) {
|
||||
logger.debug("Master key found in ${config.inputBootstrapPath}. This key will be used for decryption operations.")
|
||||
logger.debug("Root key found in ${config.inputBootstrapPath}. This key will be used for decryption operations.")
|
||||
} else {
|
||||
logger.warn("Bootstrap Conf flag present, but master key could not be found in ${config.inputBootstrapPath}.")
|
||||
logger.warn("Bootstrap Conf flag present, but root key could not be found in ${config.inputBootstrapPath}.")
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -133,7 +133,7 @@ class NiFiRegistryMode implements ToolMode {
|
||||
|
||||
if (config.writingKeyToBootstrap) {
|
||||
BootstrapUtil.writeKeyToBootstrapFile(config.encryptionKey, BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY, config.outputBootstrapPath, config.inputBootstrapPath)
|
||||
logger.info("Updated bootstrap config file with master key: ${config.outputBootstrapPath}")
|
||||
logger.info("Updated bootstrap config file with root key: ${config.outputBootstrapPath}")
|
||||
}
|
||||
|
||||
if (config.handlingNiFiRegistryProperties) {
|
||||
@ -201,11 +201,11 @@ class NiFiRegistryMode implements ToolMode {
|
||||
cli.b(longOpt: 'bootstrapConf',
|
||||
args: 1,
|
||||
argName: 'file',
|
||||
'The bootstrap.conf file containing no master key or an existing master key. If a new password or key is specified (using -p or -k) and no output bootstrap.conf file is specified, then this file will be overwritten to persist the new master key.')
|
||||
'The bootstrap.conf file containing no root key or an existing root key. If a new password or key is specified (using -p or -k) and no output bootstrap.conf file is specified, then this file will be overwritten to persist the new master key.')
|
||||
cli.B(longOpt: 'outputBootstrapConf',
|
||||
args: 1,
|
||||
argName: 'file',
|
||||
'The destination bootstrap.conf file to persist master key. If specified, the input bootstrap.conf will not be modified.')
|
||||
'The destination bootstrap.conf file to persist root key. If specified, the input bootstrap.conf will not be modified.')
|
||||
|
||||
// Options for input/output nifi-registry.properties files
|
||||
cli.r(longOpt: 'nifiRegistryProperties',
|
||||
@ -288,7 +288,7 @@ class NiFiRegistryMode implements ToolMode {
|
||||
// Determine key for encryption (required)
|
||||
determineEncryptionKey()
|
||||
if (!encryptionKey) {
|
||||
throw new RuntimeException("Failed to configure tool, could not determine encryption key. Must provide -p, -k, or -b. If using -b, bootstrap.conf argument must already contain master key.")
|
||||
throw new RuntimeException("Failed to configure tool, could not determine encryption key. Must provide -p, -k, or -b. If using -b, bootstrap.conf argument must already contain root key.")
|
||||
}
|
||||
encryptionProvider = new AESSensitivePropertyProvider(encryptionKey)
|
||||
|
||||
@ -360,11 +360,11 @@ class NiFiRegistryMode implements ToolMode {
|
||||
}
|
||||
encryptionKey = ToolUtilities.determineKey(TextDevices.defaultTextDevice(), keyHex, password, usingPassword)
|
||||
} else if (rawOptions.b) {
|
||||
logger.debug("Attempting to read master key from input bootstrap.conf file.")
|
||||
logger.debug("Attempting to read root key from input bootstrap.conf file.")
|
||||
usingBootstrapKey = true
|
||||
encryptionKey = BootstrapUtil.extractKeyFromBootstrapFile(inputBootstrapPath, BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY)
|
||||
if (!encryptionKey) {
|
||||
logger.warn("-b specified without -p or -k, but the input bootstrap.conf file did not contain a master key.")
|
||||
logger.warn("-b specified without -p or -k, but the input bootstrap.conf file did not contain a root key.")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -21,12 +21,12 @@ import org.slf4j.LoggerFactory
|
||||
|
||||
class BootstrapUtil {
|
||||
|
||||
static final String NIFI_BOOTSTRAP_KEY_PROPERTY = "nifi.bootstrap.sensitive.key";
|
||||
static final String REGISTRY_BOOTSTRAP_KEY_PROPERTY = "nifi.registry.bootstrap.sensitive.key";
|
||||
static final String NIFI_BOOTSTRAP_KEY_PROPERTY = "nifi.bootstrap.sensitive.key"
|
||||
static final String REGISTRY_BOOTSTRAP_KEY_PROPERTY = "nifi.registry.bootstrap.sensitive.key"
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(BootstrapUtil.class)
|
||||
|
||||
private static final String BOOTSTRAP_KEY_COMMENT = "# Master key in hexadecimal format for encrypted sensitive configuration values"
|
||||
private static final String BOOTSTRAP_KEY_COMMENT = "# Root key in hexadecimal format for encrypted sensitive configuration values"
|
||||
|
||||
/**
|
||||
* Tries to load keyHex from input bootstrap.conf
|
||||
@ -58,11 +58,11 @@ class BootstrapUtil {
|
||||
|
||||
|
||||
} catch (IOException e) {
|
||||
logger.error("Encountered an exception reading the master key from the input bootstrap.conf file: ${e.getMessage()}")
|
||||
logger.error("Encountered an exception reading the root key from the input bootstrap.conf file: ${e.getMessage()}")
|
||||
throw e
|
||||
}
|
||||
|
||||
return keyValue;
|
||||
return keyValue
|
||||
|
||||
}
|
||||
|
||||
@ -91,7 +91,7 @@ class BootstrapUtil {
|
||||
// Write the updated values to the output file
|
||||
outputBootstrapConfFile.text = lines.join("\n")
|
||||
} catch (IOException e) {
|
||||
logger.error("Encountered an exception reading the master key from the input bootstrap.conf file: ${e.getMessage()}")
|
||||
logger.error("Encountered an exception reading the root key from the input bootstrap.conf file: ${e.getMessage()}")
|
||||
throw e
|
||||
}
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ class ToolUtilities {
|
||||
* @param usingPassword
|
||||
* @return
|
||||
*/
|
||||
public static String determineKey(TextDevice device = TextDevices.defaultTextDevice(), String keyHex, String password, boolean usingPassword) {
|
||||
static String determineKey(TextDevice device = TextDevices.defaultTextDevice(), String keyHex, String password, boolean usingPassword) {
|
||||
if (usingPassword) {
|
||||
if (!password) {
|
||||
logger.debug("Reading password from secure console")
|
||||
@ -103,7 +103,7 @@ class ToolUtilities {
|
||||
}
|
||||
|
||||
private static String readKeyFromConsole(TextDevice textDevice) {
|
||||
textDevice.printf("Enter the master key in hexadecimal format (spaces acceptable): ")
|
||||
textDevice.printf("Enter the root key in hexadecimal format (spaces acceptable): ")
|
||||
new String(textDevice.readPassword())
|
||||
}
|
||||
|
||||
@ -133,7 +133,7 @@ class ToolUtilities {
|
||||
*
|
||||
* @return 128 , [192, 256]
|
||||
*/
|
||||
public static List<Integer> getValidKeyLengths() {
|
||||
static List<Integer> getValidKeyLengths() {
|
||||
Cipher.getMaxAllowedKeyLength("AES") > 128 ? [128, 192, 256] : [128]
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,6 @@ import org.apache.nifi.toolkit.tls.commandLine.CommandLineParseException
|
||||
import org.apache.nifi.util.NiFiProperties
|
||||
import org.apache.nifi.util.console.TextDevice
|
||||
import org.apache.nifi.util.console.TextDevices
|
||||
import org.apache.nifi.util.file.FileUtils
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider
|
||||
import org.junit.After
|
||||
import org.junit.AfterClass
|
||||
@ -994,7 +993,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
@Test
|
||||
void testShouldWriteKeyToBootstrapConf() {
|
||||
// Arrange
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File workingFile = new File("target/tmp_bootstrap.conf")
|
||||
workingFile.delete()
|
||||
|
||||
@ -1028,7 +1027,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
@Test
|
||||
void testWriteKeyToBootstrapConfShouldHandleReadFailure() {
|
||||
// Arrange
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File workingFile = new File("target/tmp_bootstrap.conf")
|
||||
workingFile.delete()
|
||||
|
||||
@ -1058,7 +1057,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
@Test
|
||||
void testWriteKeyToBootstrapConfShouldHandleWriteFailure() {
|
||||
// Arrange
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File workingFile = new File("target/tmp_bootstrap.conf")
|
||||
workingFile.delete()
|
||||
|
||||
@ -1516,7 +1515,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -1595,7 +1594,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -1677,7 +1676,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -1780,8 +1779,8 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_master_key_password.conf" :
|
||||
"src/test/resources/bootstrap_with_master_key_password_128.conf"
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_root_key_password.conf" :
|
||||
"src/test/resources/bootstrap_with_root_key_password_128.conf"
|
||||
File originalKeyFile = new File(bootstrapPath)
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
@ -2547,7 +2546,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -2629,7 +2628,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
// Start with 128-bit encryption and go to whatever is supported on this system
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_master_key_128.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_root_key_128.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3273,7 +3272,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3355,7 +3354,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
// Start with 128-bit encryption and go to whatever is supported on this system
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_master_key_128.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_root_key_128.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3435,7 +3434,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3516,7 +3515,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3735,7 +3734,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3843,7 +3842,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -3948,7 +3947,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File passwordKeyFile = new File("src/test/resources/bootstrap_with_master_key_password_128.conf")
|
||||
File passwordKeyFile = new File("src/test/resources/bootstrap_with_root_key_password_128.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -4089,7 +4088,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
|
||||
File passwordKeyFile = new File("src/test/resources/bootstrap_with_master_key_password_128.conf")
|
||||
File passwordKeyFile = new File("src/test/resources/bootstrap_with_root_key_password_128.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -4140,7 +4139,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
// Create a series of passwords with which to encrypt the flow XML, starting with the current password
|
||||
def passwordProgression = [DEFAULT_LEGACY_SENSITIVE_PROPS_KEY] + (0..5).collect { "${FLOW_PASSWORD}${it}" }
|
||||
|
||||
// The master key is not changing
|
||||
// The root key is not changing
|
||||
AESSensitivePropertyProvider spp = new AESSensitivePropertyProvider(PASSWORD_KEY_HEX_128)
|
||||
|
||||
// Act
|
||||
@ -4764,7 +4763,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -4907,13 +4906,13 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_master_key.conf" :
|
||||
"src/test/resources/bootstrap_with_master_key_128.conf"
|
||||
File masterKeyFile = new File(bootstrapPath)
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_root_key.conf" :
|
||||
"src/test/resources/bootstrap_with_root_key_128.conf"
|
||||
File rootKeyFile = new File(bootstrapPath)
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
Files.copy(masterKeyFile.toPath(), bootstrapFile.toPath())
|
||||
Files.copy(rootKeyFile.toPath(), bootstrapFile.toPath())
|
||||
|
||||
String inputPropertiesPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/nifi_with_sensitive_properties_protected_aes.properties" :
|
||||
"src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties"
|
||||
@ -4981,13 +4980,13 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
|
||||
tmpDir.mkdirs()
|
||||
setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
|
||||
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_master_key.conf" :
|
||||
"src/test/resources/bootstrap_with_master_key_128.conf"
|
||||
File masterKeyFile = new File(bootstrapPath)
|
||||
String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_root_key.conf" :
|
||||
"src/test/resources/bootstrap_with_root_key_128.conf"
|
||||
File rootKeyFile = new File(bootstrapPath)
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
Files.copy(masterKeyFile.toPath(), bootstrapFile.toPath())
|
||||
Files.copy(rootKeyFile.toPath(), bootstrapFile.toPath())
|
||||
|
||||
String inputPropertiesPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/nifi_with_sensitive_properties_protected_aes.properties" :
|
||||
"src/test/resources/nifi_with_sensitive_properties_protected_aes_128.properties"
|
||||
|
@ -17,7 +17,6 @@
|
||||
package org.apache.nifi.toolkit.encryptconfig
|
||||
|
||||
import org.apache.nifi.properties.AESSensitivePropertyProvider
|
||||
import org.apache.nifi.properties.ConfigEncryptionTool
|
||||
import org.apache.nifi.properties.NiFiPropertiesLoader
|
||||
import org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
|
||||
import org.apache.nifi.util.NiFiProperties
|
||||
@ -35,8 +34,6 @@ import org.slf4j.LoggerFactory
|
||||
import java.nio.file.Files
|
||||
import java.security.Security
|
||||
|
||||
import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
|
||||
|
||||
@RunWith(JUnit4.class)
|
||||
class EncryptConfigMainTest extends GroovyTestCase {
|
||||
private static final Logger logger = LoggerFactory.getLogger(EncryptConfigMainTest.class)
|
||||
@ -52,7 +49,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
|
||||
}
|
||||
|
||||
setupTmpDir()
|
||||
TestUtil.setupTmpDir()
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -133,9 +130,9 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
// Arrange
|
||||
exit.expectSystemExitWithStatus(0)
|
||||
|
||||
File tmpDir = setupTmpDir()
|
||||
File tmpDir = TestUtil.setupTmpDir()
|
||||
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
|
||||
File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_root_key.conf")
|
||||
File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
|
||||
bootstrapFile.delete()
|
||||
|
||||
@ -147,7 +144,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
logger.info("Original key line from bootstrap.conf: ${originalKeyLine}")
|
||||
assert originalKeyLine == "${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}="
|
||||
|
||||
final String EXPECTED_KEY_LINE = "${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}=${KEY_HEX}"
|
||||
final String EXPECTED_KEY_LINE = "${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}=${TestUtil.KEY_HEX}"
|
||||
|
||||
// Set up the NFP file
|
||||
File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
|
||||
@ -181,10 +178,10 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
"-o", outputPropertiesFile.path,
|
||||
"-i", outputLIPFile.path,
|
||||
"-u", outputAuthorizersFile.path,
|
||||
"-k", KEY_HEX,
|
||||
"-k", TestUtil.KEY_HEX,
|
||||
"-v"]
|
||||
|
||||
AESSensitivePropertyProvider spp = new AESSensitivePropertyProvider(KEY_HEX)
|
||||
AESSensitivePropertyProvider spp = new AESSensitivePropertyProvider(TestUtil.KEY_HEX)
|
||||
|
||||
exit.checkAssertionAfterwards(new Assertion() {
|
||||
void checkAssertion() {
|
||||
@ -221,7 +218,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
|
||||
}
|
||||
lipEncryptedValues.each {
|
||||
assert spp.unprotect(it.text()) == PASSWORD
|
||||
assert spp.unprotect(it.text()) == TestUtil.PASSWORD
|
||||
}
|
||||
// Check that the comments are still there
|
||||
def lipTrimmedLines = inputLIPFile.readLines().collect { it.trim() }.findAll { it }
|
||||
@ -245,7 +242,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
|
||||
}
|
||||
authorizersEncryptedValues.each {
|
||||
assert spp.unprotect(it.text()) == PASSWORD
|
||||
assert spp.unprotect(it.text()) == TestUtil.PASSWORD
|
||||
}
|
||||
// Check that the comments are still there
|
||||
def authorizersTrimmedLines = inputAuthorizersFile.readLines().collect { it.trim() }.findAll { it }
|
||||
@ -281,5 +278,4 @@ class EncryptConfigMainTest extends GroovyTestCase {
|
||||
|
||||
// Assertions defined above
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,7 +16,7 @@
|
||||
*/
|
||||
package org.apache.nifi.toolkit.encryptconfig
|
||||
|
||||
import groovy.util.slurpersupport.GPathResult
|
||||
|
||||
import org.apache.commons.lang3.SystemUtils
|
||||
import org.apache.nifi.properties.AESSensitivePropertyProvider
|
||||
import org.apache.nifi.toolkit.encryptconfig.util.NiFiRegistryAuthorizersXmlEncryptor
|
||||
@ -29,10 +29,10 @@ import java.nio.file.attribute.PosixFilePermission
|
||||
class TestUtil {
|
||||
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT = absolutePathForResource('/nifi-registry/bootstrap_default.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY = absolutePathForResource('/nifi-registry/bootstrap_without_master_key.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY = absolutePathForResource('/nifi-registry/bootstrap_with_empty_master_key.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_128 = absolutePathForResource('/nifi-registry/bootstrap_with_master_key_128.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128 = absolutePathForResource('/nifi-registry/bootstrap_with_master_key_from_password_128.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY = absolutePathForResource('/nifi-registry/bootstrap_without_root_key.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY = absolutePathForResource('/nifi-registry/bootstrap_with_empty_root_key.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_128 = absolutePathForResource('/nifi-registry/bootstrap_with_root_key_128.conf')
|
||||
static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128 = absolutePathForResource('/nifi-registry/bootstrap_with_root_key_from_password_128.conf')
|
||||
|
||||
static final String RESOURCE_REGISTRY_PROPERTIES_COMMENTED = absolutePathForResource('/nifi-registry/nifi-registry-commented.properties')
|
||||
static final String RESOURCE_REGISTRY_PROPERTIES_EMPTY = absolutePathForResource('/nifi-registry/nifi-registry-empty.properties')
|
||||
|
@ -16,12 +16,11 @@
|
||||
*/
|
||||
package org.apache.nifi.toolkit.encryptconfig.util
|
||||
|
||||
import org.apache.nifi.toolkit.encryptconfig.TestUtil
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
import spock.lang.Specification
|
||||
|
||||
import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
|
||||
|
||||
class BootstrapUtilSpec extends Specification {
|
||||
private static final Logger logger = LoggerFactory.getLogger(BootstrapUtilSpec.class)
|
||||
|
||||
@ -33,12 +32,12 @@ class BootstrapUtilSpec extends Specification {
|
||||
|
||||
// runs before the first feature method
|
||||
def setupSpec() {
|
||||
setupTmpDir()
|
||||
TestUtil.setupTmpDir()
|
||||
}
|
||||
|
||||
// runs after the last feature method
|
||||
def cleanupSpec() {
|
||||
cleanupTmpDir()
|
||||
TestUtil.cleanupTmpDir()
|
||||
}
|
||||
|
||||
def "test extractKeyFromBootstrapFile with Registry bootstrap.conf"() {
|
||||
@ -48,24 +47,24 @@ class BootstrapUtilSpec extends Specification {
|
||||
|
||||
|
||||
when: "bootstrap.conf has no key property"
|
||||
def actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY, bootstrapKeyProperty)
|
||||
def actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY, bootstrapKeyProperty)
|
||||
|
||||
then: "null is returned"
|
||||
actualKeyHex == null
|
||||
|
||||
|
||||
when: "bootstrap.conf has an empty key property"
|
||||
actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY, bootstrapKeyProperty)
|
||||
actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY, bootstrapKeyProperty)
|
||||
|
||||
then: "null is returned"
|
||||
actualKeyHex == null
|
||||
|
||||
|
||||
when: "bootstrap.conf has a populated key property"
|
||||
actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, bootstrapKeyProperty)
|
||||
actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, bootstrapKeyProperty)
|
||||
|
||||
then: "key is returned"
|
||||
actualKeyHex == KEY_HEX_128
|
||||
actualKeyHex == TestUtil.KEY_HEX_128
|
||||
|
||||
|
||||
when: "bootstrap.conf file does not exist"
|
||||
@ -80,34 +79,32 @@ class BootstrapUtilSpec extends Specification {
|
||||
|
||||
setup:
|
||||
def bootstrapKeyProperty = BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY
|
||||
def outFile1 = generateTmpFilePath()
|
||||
def outFile2 = generateTmpFilePath()
|
||||
def outFile3 = generateTmpFilePath()
|
||||
def expected = RESOURCE_REGISTRY_BOOTSTRAP_KEY_128
|
||||
def outFile1 = TestUtil.generateTmpFilePath()
|
||||
def outFile2 = TestUtil.generateTmpFilePath()
|
||||
def outFile3 = TestUtil.generateTmpFilePath()
|
||||
def expected = TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_KEY_128
|
||||
|
||||
|
||||
when: "input is default bootstrap.conf"
|
||||
BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile1, RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
|
||||
BootstrapUtil.writeKeyToBootstrapFile(TestUtil.KEY_HEX_128, bootstrapKeyProperty, outFile1, TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
|
||||
|
||||
then: "output file content matches populated bootstrap file"
|
||||
assertBootstrapFilesAreEqual(expected, outFile1, true)
|
||||
TestUtil.assertBootstrapFilesAreEqual(expected, outFile1, true)
|
||||
and: "key is readable from output file"
|
||||
BootstrapUtil.extractKeyFromBootstrapFile(outFile1, bootstrapKeyProperty) == KEY_HEX_128
|
||||
BootstrapUtil.extractKeyFromBootstrapFile(outFile1, bootstrapKeyProperty) == TestUtil.KEY_HEX_128
|
||||
|
||||
|
||||
when: "input bootstrap.conf has no key property"
|
||||
BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile2, RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY)
|
||||
BootstrapUtil.writeKeyToBootstrapFile(TestUtil.KEY_HEX_128, bootstrapKeyProperty, outFile2, TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY)
|
||||
|
||||
then: "output file content matches pre-populated bootstrap file"
|
||||
assertBootstrapFilesAreEqual(expected, outFile2, true)
|
||||
TestUtil.assertBootstrapFilesAreEqual(expected, outFile2, true)
|
||||
|
||||
|
||||
when: "input bootstrap.conf has existing, different master key"
|
||||
BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile3, RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128)
|
||||
when: "input bootstrap.conf has existing, different root key"
|
||||
BootstrapUtil.writeKeyToBootstrapFile(TestUtil.KEY_HEX_128, bootstrapKeyProperty, outFile3, TestUtil.RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128)
|
||||
|
||||
then: "output file content matches pre-populated bootstrap file"
|
||||
assertBootstrapFilesAreEqual(expected, outFile3, true)
|
||||
|
||||
TestUtil.assertBootstrapFilesAreEqual(expected, outFile3, true)
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=
|
||||
|
||||
###
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210
|
||||
|
||||
###
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA9876543210
|
||||
|
||||
###
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=2C576A9585DB862F5ECBEE5B4FFFCCA14B18D8365968D7081651006507AD2BDE
|
||||
|
||||
###
|
@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
|
||||
#Set headless mode by default
|
||||
java.arg.14=-Djava.awt.headless=true
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.bootstrap.sensitive.key=2C576A9585DB862F5ECBEE5B4FFFCCA1
|
||||
|
||||
###
|
@ -44,5 +44,5 @@ java.arg.4=-Djava.net.preferIPv4Stack=true
|
||||
java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
|
||||
java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.registry.bootstrap.sensitive.key=
|
@ -44,5 +44,5 @@ java.arg.4=-Djava.net.preferIPv4Stack=true
|
||||
java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
|
||||
java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.registry.bootstrap.sensitive.key=
|
@ -44,5 +44,5 @@ java.arg.4=-Djava.net.preferIPv4Stack=true
|
||||
java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
|
||||
java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.registry.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA9876543210
|
@ -44,5 +44,5 @@ java.arg.4=-Djava.net.preferIPv4Stack=true
|
||||
java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
|
||||
java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
|
||||
|
||||
# Master key in hexadecimal format for encrypted sensitive configuration values
|
||||
# Root key in hexadecimal format for encrypted sensitive configuration values
|
||||
nifi.registry.bootstrap.sensitive.key=2C576A9585DB862F5ECBEE5B4FFFCCA1
|
Loading…
x
Reference in New Issue
Block a user