You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2021/08/13 15:57:22 UTC

[nifi] branch main updated: NIFI-6615 Added Azure Key Vault Key Sensitive Property Provider

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 714670b  NIFI-6615 Added Azure Key Vault Key Sensitive Property Provider
714670b is described below

commit 714670b8e660772613184d5f64c09936def71268
Author: Emilio Setiadarma <em...@gmail.com>
AuthorDate: Tue Aug 10 17:08:55 2021 -0700

    NIFI-6615 Added Azure Key Vault Key Sensitive Property Provider
    
    This closes #5274
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 nifi-assembly/NOTICE                               |  11 +
 .../nifi/properties/BootstrapProperties.java       |   1 +
 .../nifi-sensitive-property-provider/pom.xml       |  47 ++++
 ...r.java => AWSKMSSensitivePropertyProvider.java} |   6 +-
 .../AzureKeyVaultKeySensitivePropertyProvider.java | 297 +++++++++++++++++++++
 .../nifi/properties/PropertyProtectionScheme.java  |   1 +
 .../StandardSensitivePropertyProviderFactory.java  |   4 +-
 ...java => AWSKMSSensitivePropertyProviderIT.java} |  10 +-
 ...ureKeyVaultKeySensitivePropertyProviderIT.java} |  80 +++---
 .../src/main/asciidoc/administration-guide.adoc    | 243 +++++++++--------
 nifi-docs/src/main/asciidoc/toolkit-guide.adoc     | 141 +++++-----
 .../src/main/resources/conf/bootstrap-azure.conf   |  20 ++
 .../src/main/resources/conf/bootstrap.conf         |   3 +
 .../src/main/resources/conf/bootstrap-azure.conf   |  20 ++
 .../src/main/resources/conf/bootstrap.conf         |   3 +
 15 files changed, 650 insertions(+), 237 deletions(-)

diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 1b5cfcb..25ae170 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -2093,6 +2093,17 @@ The following binary components are provided under the Eclipse Distribution Lice
     (EDL 1.0) Jakarta Activation (com.sun.activation:jakarta.activation:jar:2.0.1)
     (EDL 1.0) Jakarta XML Binding API (jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.2)
 
+************************
+The MIT License
+************************
+
+   (MIT) Azure SDK for Java
+     The following NOTICE information applies:
+       Copyright 2021 Microsoft Corporation or its affiliates. All Rights Reserved.
+
+       This product includes software developed by
+       Microsoft Corporation (https://www.microsoft.com/).
+
 *****************
 Mozilla Public License v2.0
 *****************
diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
index 830ba05..d38b747 100644
--- a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
@@ -35,6 +35,7 @@ public class BootstrapProperties extends StandardReadableProperties {
         SENSITIVE_KEY("bootstrap.sensitive.key"),
         HASHICORP_VAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.hashicorp.vault.conf"),
         AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.kms.conf"),
+        AZURE_KEYVAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.azure.keyvault.conf"),
         CONTEXT_MAPPING_PREFIX("bootstrap.protection.context.mapping.");
 
         private final String key;
diff --git a/nifi-commons/nifi-sensitive-property-provider/pom.xml b/nifi-commons/nifi-sensitive-property-provider/pom.xml
index 9a0bf77..d680559 100644
--- a/nifi-commons/nifi-sensitive-property-provider/pom.xml
+++ b/nifi-commons/nifi-sensitive-property-provider/pom.xml
@@ -60,6 +60,53 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-security-keyvault-keys</artifactId>
+            <version>4.3.1</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.azure</groupId>
+                    <artifactId>azure-core-http-netty</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.woodstox</groupId>
+                    <artifactId>woodstox-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.woodstox</groupId>
+                    <artifactId>stax2-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>io.netty</groupId>
+                    <artifactId>netty-tcnative-boringssl-static</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-identity</artifactId>
+            <version>1.3.4</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.azure</groupId>
+                    <artifactId>azure-core-http-netty</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.woodstox</groupId>
+                    <artifactId>woodstox-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.woodstox</groupId>
+                    <artifactId>stax2-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-core-http-okhttp</artifactId>
+            <version>1.7.1</version>
+        </dependency>
+        <dependency>
             <groupId>software.amazon.awssdk</groupId>
             <artifactId>url-connection-client</artifactId>
             <version>${aws.sdk.version}</version>
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
similarity index 98%
rename from nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
rename to nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
index 860d5c9..3386488 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
@@ -44,8 +44,8 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.Paths;
 import java.util.Objects;
 
-public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
-    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+public class AWSKMSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSKMSSensitivePropertyProvider.class);
 
     private static final String AWS_PREFIX = "aws";
     private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
@@ -60,7 +60,7 @@ public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvi
     private String keyId;
 
 
-    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+    AWSKMSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
         super(bootstrapProperties);
         Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
         awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProvider.java
new file mode 100644
index 0000000..a5adf1a
--- /dev/null
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProvider.java
@@ -0,0 +1,297 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.azure.core.exception.ResourceNotFoundException;
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.security.keyvault.keys.cryptography.CryptographyClient;
+import com.azure.security.keyvault.keys.cryptography.CryptographyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.DecryptResult;
+import com.azure.security.keyvault.keys.cryptography.models.EncryptResult;
+import com.azure.security.keyvault.keys.cryptography.models.EncryptionAlgorithm;
+import com.azure.security.keyvault.keys.models.KeyOperation;
+import com.azure.security.keyvault.keys.models.KeyProperties;
+
+import java.util.Base64;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Objects;
+
+public class AzureKeyVaultKeySensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AzureKeyVaultKeySensitivePropertyProvider.class);
+
+    private static final String AZURE_PREFIX = "azure";
+    private static final String KEYVAULT_KEY_PROPS_NAME = "azure.keyvault.key.id";
+    private static final String ENCRYPTION_ALGORITHM_PROPS_NAME = "azure.keyvault.encryption.algorithm";
+
+    private static final Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties azureBootstrapProperties;
+    private CryptographyClient client;
+    private String keyId;
+    private String algorithm;
+
+    AzureKeyVaultKeySensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "Bootstrap Properties required");
+        azureBootstrapProperties = getAzureBootstrapProperties(bootstrapProperties);
+        loadRequiredAzureProperties(azureBootstrapProperties);
+    }
+
+    /**
+     * Initializes the Azure Key Vault Cryptography Client to be used for encrypt, decrypt and other interactions with Azure Key Vault.
+     * Uses the default Azure credentials provider chain.
+     */
+    private void initializeClient() {
+        if (azureBootstrapProperties == null) {
+            logger.warn("Azure Bootstrap Properties are required for Key Vault Client initialization");
+            return;
+        }
+
+        if (StringUtils.isBlank(keyId)) {
+            logger.warn("Cannot initialize client if Azure Key Vault Key ID is blank");
+            return;
+        }
+
+        try {
+            client = new CryptographyClientBuilder()
+                    .credential(new DefaultAzureCredentialBuilder().build())
+                    .keyIdentifier(keyId)
+                    .buildClient();
+        } catch (final RuntimeException e) {
+            throw new SensitivePropertyProtectionException("Azure Key Vault Client initialization failed", e);
+        }
+    }
+
+    /**
+     * Validates the key provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials provided
+     * during the initialization of the client.
+     */
+    private void validate() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("Azure Key Vault validation failed: Client not initialized");
+        }
+
+        if (StringUtils.isBlank(keyId)) {
+            throw new SensitivePropertyProtectionException("Azure Key Vault validation failed: Key not specified");
+        }
+
+        try {
+            final KeyProperties keyProps = client.getKey().getProperties();
+            if (!keyProps.isEnabled()) {
+                throw new SensitivePropertyProtectionException("Azure Key Vault validation failed: Key not enabled");
+            }
+
+            final List<KeyOperation> keyOps = client.getKey().getKeyOperations();
+            if (!(keyOps.contains(KeyOperation.ENCRYPT) && keyOps.contains(KeyOperation.DECRYPT))) {
+                throw new SensitivePropertyProtectionException("Azure Key Vault validation failed: Encrypt and Decrypt not supported");
+            }
+        } catch (final ResourceNotFoundException e) {
+            throw new SensitivePropertyProtectionException("Azure Key Vault validation failed: Key not found", e);
+        } catch (final RuntimeException e) {
+            throw new SensitivePropertyProtectionException("Azure Key Vault validation failed", e);
+        }
+    }
+
+    /**
+     * Checks if we have the required properties {@link #keyId} and {@link #algorithm} from bootstrap-azure.conf
+     * for Azure KeyVault and loads it into the appropriate variables, will load null if values don't exist.
+     * Note: This function does not verify if the properties are valid.
+     * @param props the properties representing bootstrap-azure.conf
+     */
+    private void loadRequiredAzureProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KEYVAULT_KEY_PROPS_NAME);
+            algorithm = props.getProperty(ENCRYPTION_ALGORITHM_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AZURE_KEYVAULT_SENSITIVE_PROPERTY_PROVIDER_CONF property is configured to the
+     * bootstrap-azure.conf file. Also will load bootstrap-azure.conf to {@link #azureBootstrapProperties} if possible
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-azure.conf, null otherwise
+     */
+    private BootstrapProperties getAzureBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-azure.conf file based on path specified in
+        // "nifi.bootstrap.protection.azure.keyvault.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AZURE_KEYVAULT_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("Azure Key Vault properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AZURE_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks the BootstrapProperties corresponding to bootstrap-azure.conf for the required configurations
+     * for Azure encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     * Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return True if bootstrap-azure.conf contains the required properties for Azure SPP, False otherwise
+     */
+    private boolean hasRequiredAzureProperties() {
+        return azureBootstrapProperties != null && StringUtils.isNoneBlank(keyId, algorithm);
+    }
+
+    /**
+     * Return true if this SensitivePropertyProvider is supported, given the provided Bootstrap properties.
+     * @return True if this SensitivePropertyProvider is supported
+     */
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAzureProperties();
+    }
+
+    /**
+     * Return the appropriate PropertyProtectionScheme for this provider.
+     *
+     * @return The PropertyProtectionScheme
+     */
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AZURE_KEYVAULT_KEY;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AZURE_KEYVAULT_KEY.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AZURE_KEYVAULT_KEY.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext of this value encrypted using a key stored in Azure Key Vault.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file
+     */
+    private byte[] encrypt(final byte[] input) {
+        EncryptResult encryptResult = client.encrypt(EncryptionAlgorithm.fromString(algorithm), input);
+        return encryptResult.getCipherText();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext decrypted using a key stored in Azure Key Vault
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application
+     */
+    private byte[] decrypt(final byte[] input) {
+        DecryptResult decryptResult = client.decrypt(EncryptionAlgorithm.fromString(algorithm), input);
+        return decryptResult.getPlainText();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the configuration required for Azure Key Vault.
+     */
+    private void checkAndInitializeClient() {
+        if (client == null) {
+            initializeClient();
+            validate();
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * Encrypts a sensitive value using a key managed by Azure Key Vault.
+     *
+     * @param unprotectedValue the sensitive value
+     * @param context The context of the value (ignored in this implementation)
+     * @return the value to persist in the {@code nifi.properties} file
+     */
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.getEncoder().encodeToString(cipherBytes);
+        } catch (final RuntimeException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * Decrypts a secured value from a ciphertext using a key managed by Azure Key Vault.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file
+     * @param context The context of the value (ignored in this implementation)
+     * @return the raw value to be used by the application
+     */
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.getDecoder().decode(protectedValue);
+            final byte[] plainBytes = decrypt(cipherBytes);
+            return new String(plainBytes, PROPERTY_CHARSET);
+        } catch (final RuntimeException e) {
+            throw new SensitivePropertyProtectionException("Decrypt failed", e);
+        }
+    }
+
+    /**
+     * Nothing required to be done for Azure Client cleanUp function.
+     */
+    @Override
+    public void cleanUp() {}
+}
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
index ee3859b..1eb565b 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
@@ -26,6 +26,7 @@ import java.util.Objects;
 public enum PropertyProtectionScheme {
     AES_GCM("aes/gcm/(128|192|256)", "aes/gcm/%s", "AES Sensitive Property Provider", true),
     AWS_KMS("aws/kms", "aws/kms", "AWS KMS Sensitive Property Provider", false),
+    AZURE_KEYVAULT_KEY("azure/keyvault/key", "azure/keyvault/key", "Azure Key Vault Key Sensitive Property Provider", false),
     HASHICORP_VAULT_KV("hashicorp/vault/kv/[a-zA-Z0-9_-]+", "hashicorp/vault/kv/%s", "HashiCorp Vault Key/Value Engine Sensitive Property Provider", false),
     HASHICORP_VAULT_TRANSIT("hashicorp/vault/transit/[a-zA-Z0-9_-]+", "hashicorp/vault/transit/%s", "HashiCorp Vault Transit Engine Sensitive Property Provider", false);
 
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
index 9ba0178..4439d74 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
@@ -121,7 +121,9 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
             case AES_GCM:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new AESSensitivePropertyProvider(keyHex));
             case AWS_KMS:
-                return providerMap.computeIfAbsent(protectionScheme, s -> new AWSSensitivePropertyProvider(getBootstrapProperties()));
+                return providerMap.computeIfAbsent(protectionScheme, s -> new AWSKMSSensitivePropertyProvider(getBootstrapProperties()));
+            case AZURE_KEYVAULT_KEY:
+                return providerMap.computeIfAbsent(protectionScheme, s -> new AzureKeyVaultKeySensitivePropertyProvider(getBootstrapProperties()));
             case HASHICORP_VAULT_TRANSIT:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new HashiCorpVaultTransitSensitivePropertyProvider(getBootstrapProperties()));
             case HASHICORP_VAULT_KV:
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
similarity index 94%
copy from nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
copy to nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
index 8ba8bfb..a22945c 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
@@ -52,8 +52,8 @@ import java.util.Properties;
  *
  */
 
-public class AWSSensitivePropertyProviderIT {
-    private static final String SAMPLE_PLAINTEXT = "AWSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+public class AWSKMSSensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "AWSKMSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
     private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
     private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
     private static final String REGION_KEY_PROPS_NAME = "aws.region";
@@ -63,13 +63,13 @@ public class AWSSensitivePropertyProviderIT {
 
     private static final String EMPTY_PROPERTY = "";
 
-    private static AWSSensitivePropertyProvider spp;
+    private static AWSKMSSensitivePropertyProvider spp;
 
     private static BootstrapProperties props;
 
     private static Path mockBootstrapConf, mockAWSBootstrapConf;
 
-    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProviderIT.class);
+    private static final Logger logger = LoggerFactory.getLogger(AWSKMSSensitivePropertyProviderIT.class);
 
     private static void initializeBootstrapProperties() throws IOException{
         mockBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
@@ -100,7 +100,7 @@ public class AWSSensitivePropertyProviderIT {
     public static void initOnce() throws IOException {
         initializeBootstrapProperties();
         Assert.assertNotNull(props);
-        spp = new AWSSensitivePropertyProvider(props);
+        spp = new AWSKMSSensitivePropertyProvider(props);
         Assert.assertNotNull(spp);
     }
 
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProviderIT.java
similarity index 50%
rename from nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
rename to nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProviderIT.java
index 8ba8bfb..c89f872 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AzureKeyVaultKeySensitivePropertyProviderIT.java
@@ -31,50 +31,48 @@ import java.nio.file.Path;
 import java.util.Properties;
 
 /**
- * To run this test, make sure to first configure sensitive credential information as in the following link
- * https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html
+ * For the purposes of running this test, if you have a key, first make sure you have configured
+ * credential information as in the following link
+ * https://docs.microsoft.com/en-us/java/api/overview/azure/security-keyvault-keys-readme?view=azure-java-stable
  *
- * If you don't have a key then run:
- * aws kms create-key
+ * Then simply set the system property -Dazure.keyvault.key.id to the key identifier
+ * when running the integration test.
+ * The key identifier takes the form: "https://keyvault-name.vault.azure.net/keys/key-name/keyID"
  *
- * Take note of the key id or arn.
+ * Otherwise, to create a key, it will be simplest through the Azure web portal. First create a resource group,
+ * then a key vault tied to that resource group. Then you can create a key through the key vault.
  *
- * Then, set the system property -Daws.kms.key.id to the either key id value or arn value
- *
- * The following settings are optional. If you have a default AWS configuration and credentials in ~/.aws then
- * it will take that. Otherwise you can set all of the following:
- * set the system property -Daws.access.key.id to the access key id
- * set the system property -Daws.secret.access.key to the secret access key
- * set the system property -Daws.region to the region
- *
- * After you are satisfied with the test, and you don't need the key, you may schedule key deletion with:
- * aws kms schedule-key-deletion --key-id "key id" --pending-window-in-days "number of days"
+ * Then simply set the system property -Dazure.keyvault.key.id to the key identifier
+ * when running the integration test.
+ * The key identifier takes the form: "https://keyvault-name.vault.azure.net/keys/key-name/keyID"
  *
+ * Another system property that needs to be set is the algorithm: set it as
+ * -Dazure.keyvault.encryption.algorithm=algorithm
+ * Available algorithms can be found in:
+ * https://docs.microsoft.com/en-us/java/api/com.azure.security.keyvault.keys.cryptography.models.encryptionalgorithm
  */
 
-public class AWSSensitivePropertyProviderIT {
-    private static final String SAMPLE_PLAINTEXT = "AWSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
-    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
-    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
-    private static final String REGION_KEY_PROPS_NAME = "aws.region";
-    private static final String KMS_KEY_PROPS_NAME = "aws.kms.key.id";
+public class AzureKeyVaultKeySensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "AzureKeyVaultKeySensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+    private static final String KEYVAULT_KEY_PROPS_NAME = "azure.keyvault.key.id";
+    private static final String ENCRYPTION_ALGORITHM_PROPS_NAME = "azure.keyvault.encryption.algorithm";
 
-    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.kms.conf";
+    private static final String BOOTSTRAP_AZURE_FILE_PROPS_NAME = "nifi.bootstrap.protection.azure.keyvault.conf";
 
     private static final String EMPTY_PROPERTY = "";
 
-    private static AWSSensitivePropertyProvider spp;
+    private static AzureKeyVaultKeySensitivePropertyProvider spp;
 
     private static BootstrapProperties props;
 
-    private static Path mockBootstrapConf, mockAWSBootstrapConf;
+    private static Path mockBootstrapConf, mockAzureBootstrapConf;
 
-    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProviderIT.class);
+    private static final Logger logger = LoggerFactory.getLogger(AzureKeyVaultKeySensitivePropertyProviderIT.class);
 
-    private static void initializeBootstrapProperties() throws IOException{
+    private static void initializeBootstrapProperties() throws IOException {
         mockBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
-        mockAWSBootstrapConf = Files.createTempFile("bootstrap-aws", ".conf").toAbsolutePath();
-        IOUtil.writeText(BOOTSTRAP_AWS_FILE_PROPS_NAME + "=" + mockAWSBootstrapConf.toAbsolutePath(), mockBootstrapConf.toFile());
+        mockAzureBootstrapConf = Files.createTempFile("bootstrap-azure", ".conf").toAbsolutePath();
+        IOUtil.writeText(BOOTSTRAP_AZURE_FILE_PROPS_NAME + "=" + mockAzureBootstrapConf.toAbsolutePath(), mockBootstrapConf.toFile());
 
         final Properties bootstrapProperties = new Properties();
         try (final InputStream inputStream = Files.newInputStream(mockBootstrapConf)) {
@@ -82,41 +80,37 @@ public class AWSSensitivePropertyProviderIT {
             props = new BootstrapProperties("nifi", bootstrapProperties, mockBootstrapConf);
         }
 
-        String accessKey = System.getProperty(ACCESS_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String secretKey = System.getProperty(SECRET_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String region = System.getProperty(REGION_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String keyId = System.getProperty(KMS_KEY_PROPS_NAME, EMPTY_PROPERTY);
+        String keyId = System.getProperty(KEYVAULT_KEY_PROPS_NAME, EMPTY_PROPERTY);
+        String algorithm = System.getProperty(ENCRYPTION_ALGORITHM_PROPS_NAME, EMPTY_PROPERTY);
 
         StringBuilder bootstrapConfText = new StringBuilder();
-        bootstrapConfText.append(ACCESS_KEY_PROPS_NAME + "=" + accessKey);
-        bootstrapConfText.append("\n" + SECRET_KEY_PROPS_NAME + "=" + secretKey);
-        bootstrapConfText.append("\n" + REGION_KEY_PROPS_NAME + "=" + region);
-        bootstrapConfText.append("\n" + KMS_KEY_PROPS_NAME + "=" + keyId);
-
-        IOUtil.writeText(bootstrapConfText.toString(), mockAWSBootstrapConf.toFile());
+        String lineSeparator = System.getProperty("line.separator");
+        bootstrapConfText.append(KEYVAULT_KEY_PROPS_NAME + "=" + keyId);
+        bootstrapConfText.append(lineSeparator + ENCRYPTION_ALGORITHM_PROPS_NAME + "=" + algorithm);
+        IOUtil.writeText(bootstrapConfText.toString(), mockAzureBootstrapConf.toFile());
     }
 
     @BeforeClass
     public static void initOnce() throws IOException {
         initializeBootstrapProperties();
         Assert.assertNotNull(props);
-        spp = new AWSSensitivePropertyProvider(props);
+        spp = new AzureKeyVaultKeySensitivePropertyProvider(props);
         Assert.assertNotNull(spp);
     }
 
     @AfterClass
     public static void tearDownOnce() throws IOException {
         Files.deleteIfExists(mockBootstrapConf);
-        Files.deleteIfExists(mockAWSBootstrapConf);
+        Files.deleteIfExists(mockAzureBootstrapConf);
 
         spp.cleanUp();
     }
 
     @Test
     public void testEncryptDecrypt() {
-        logger.info("Running testEncryptDecrypt of AWS SPP integration test");
-        runEncryptDecryptTest();
-        logger.info("testEncryptDecrypt of AWS SPP integration test completed");
+        logger.info("Running testEncryptDecrypt of Azure Key Vault Key SPP integration test");
+        this.runEncryptDecryptTest();
+        logger.info("testEncryptDecrypt of Azure Key Vault Key SPP integration test completed");
     }
 
     private static void runEncryptDecryptTest() {
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 4f71047..95fcaa3 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -671,8 +671,8 @@ NOTE: The identities configured in the Initial Admin Identity, the Node Identity
 NOTE: Any users in the legacy users file must be found in the configured User Group Provider.
 
 NOTE: Any identity mapping rules specified in _nifi.properties_ will also be applied to the node identities,
-            so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
-            in the configured User Group Provider.
+so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
+in the configured User Group Provider.
 
 ==== StandardManagedAuthorizer
 
@@ -1456,7 +1456,7 @@ To allow User2 to connect GenerateFlowFile to LogAttribute, as User1:
 1. Select the root process group. The Operate palette is updated with details for the root process group.
 2. Select the Access Policies icon (image:iconAccessPolicies.png["Access Policies Icon"]) from the Operate palette and the Access Policies dialog opens.
 3. Select "modify the component” from the policy drop-down.
-  image:process-group-modify-policy.png["Process Group Modify Policy"]
+image:process-group-modify-policy.png["Process Group Modify Policy"]
 [start=4]
 4. Select the Add User icon (image:iconAddUser.png["Add User Icon"]). Find or enter User2 and select OK.
 
@@ -1492,7 +1492,7 @@ To allow User2 to connect GenerateFlowFile to ReplaceText, as User1:
 1. Select the root process group. The Operate palette is updated with details for the root process group.
 2. Select the Access Policies icon (image:iconAccessPolicies.png["Access Policies Icon"]).
 3. Select "view the component” from the policy drop-down.
-  image:process-group-view-policy.png["Process Group View Policy"]
+image:process-group-view-policy.png["Process Group View Policy"]
 [start=4]
 4. Select the Add User icon (image:iconAddUser.png["Add User Icon"]). Find or enter User2 and select OK.
 
@@ -1817,7 +1817,7 @@ Following are the configuration properties available inside the `bootstrap-hashi
 |===
 
 === AWS KMS provider
-This provider uses AWS Key Management Service (https://aws.amazon.com/kms/) for decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in `bootstrap.conf`. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.
+This provider uses https://aws.amazon.com/kms/[AWS Key Management Service] for decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in `bootstrap.conf`. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.
 
 ==== Required properties
 [options="header,footer"]
@@ -1836,6 +1836,17 @@ This provider uses AWS Key Management Service (https://aws.amazon.com/kms/) for
 |`aws.secret.access.key`|The secret access key used to access AWS KMS.|_none_
 |===
 
+=== Azure Key Vault Key provider
+This protection scheme uses keys managed by https://docs.microsoft.com/en-us/azure/key-vault/keys/about-keys[Azure Key Vault Keys] for encryption and decryption. Azure Key Vault configuration properties can be stored in the `bootstrap-azure.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. The provider will use the Azure default credentials provider chain as described in the https://docs.microsoft.com/en-us/java/api/overview/azure/security-keyvault-keys-readme?v [...]
+
+==== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`azure.keyvault.key.id`|The identifier of the key that the Azure Key Vault client uses for encryption and decryption.|_none_
+|`azure.keyvault.encryption.algorithm`|The encryption algorithm that the Azure Key Vault client uses for encryption and decryption.|_none_
+|===
+
 === Property Context Mapping
 Some encryption providers store protected values in an external service instead of persisting the encrypted values directly in the configuration file.  To support this use case, a property context is defined for each protected property in NiFi's configuration files, in the format: `{context-name}/{property-name}`
 
@@ -2081,37 +2092,37 @@ they must be set the same on every instance in the cluster.
 For each Node, the minimum properties to configure are as follows:
 
 * Under the _Web Properties_ section, set either the HTTP or HTTPS port that you want the Node to run on.
-  Also, consider whether you need to set the HTTP or HTTPS host property. All nodes in the cluster should use the same protocol setting.
+Also, consider whether you need to set the HTTP or HTTPS host property. All nodes in the cluster should use the same protocol setting.
 * Under the _State Management section_, set the `nifi.state.management.provider.cluster` property
-  to the identifier of the Cluster State Provider. Ensure that the Cluster State Provider has been
-  configured in the _state-management.xml_ file. See <<state_providers>> for more information.
+to the identifier of the Cluster State Provider. Ensure that the Cluster State Provider has been
+configured in the _state-management.xml_ file. See <<state_providers>> for more information.
 * Under _Cluster Node Properties_, set the following:
 ** `nifi.cluster.is.node` - Set this to _true_.
 ** `nifi.cluster.node.address` - Set this to the fully qualified hostname of the node. If left blank, it defaults to `localhost`.
 ** `nifi.cluster.node.protocol.port` - Set this to an open port that is higher than 1024 (anything lower requires root).
 ** `nifi.cluster.node.protocol.threads` - The number of threads that should be used to communicate with other nodes in the cluster. This property
-   defaults to `10`. A thread pool is used for replicating requests to all nodes, and the
-   thread pool will never have fewer than this number of threads. It will grow as needed up to the maximum value set by the `nifi.cluster.node.protocol.max.threads`
-   property.
+defaults to `10`. A thread pool is used for replicating requests to all nodes, and the
+thread pool will never have fewer than this number of threads. It will grow as needed up to the maximum value set by the `nifi.cluster.node.protocol.max.threads`
+property.
 ** `nifi.cluster.node.protocol.max.threads` - The maximum number of threads that should be used to communicate with other nodes in the cluster. This property
-	defaults to `50`. A thread pool is used for replication requests to all nodes, and the thread pool will have a "core" size that is configured by the
-	`nifi.cluster.node.protocol.threads` property. However, if necessary, the thread pool will increase the number of active threads to the limit
-	set by this property.
+defaults to `50`. A thread pool is used for replication requests to all nodes, and the thread pool will have a "core" size that is configured by the
+`nifi.cluster.node.protocol.threads` property. However, if necessary, the thread pool will increase the number of active threads to the limit
+set by this property.
 ** `nifi.zookeeper.connect.string` - The Connect String that is needed to connect to Apache ZooKeeper. This is a comma-separated list
-   of hostname:port pairs. For example, `localhost:2181,localhost:2182,localhost:2183`. This should contain a list of all ZooKeeper
-   instances in the ZooKeeper quorum.
+of hostname:port pairs. For example, `localhost:2181,localhost:2182,localhost:2183`. This should contain a list of all ZooKeeper
+instances in the ZooKeeper quorum.
 ** `nifi.zookeeper.root.node` - The root ZNode that should be used in ZooKeeper. ZooKeeper provides a directory-like structure
-   for storing data. Each 'directory' in this structure is referred to as a ZNode. This denotes the root ZNode, or 'directory',
-   that should be used for storing data. The default value is `/root`. This is important to set correctly, as which cluster
-   the NiFi instance attempts to join is determined by which ZooKeeper instance it connects to and the ZooKeeper Root Node
-   that is specified.
+for storing data. Each 'directory' in this structure is referred to as a ZNode. This denotes the root ZNode, or 'directory',
+that should be used for storing data. The default value is `/root`. This is important to set correctly, as which cluster
+the NiFi instance attempts to join is determined by which ZooKeeper instance it connects to and the ZooKeeper Root Node
+that is specified.
 ** `nifi.cluster.flow.election.max.wait.time` - Specifies the amount of time to wait before electing a Flow as the "correct" Flow.
-   If the number of Nodes that have voted is equal to the number specified by the `nifi.cluster.flow.election.max.candidates`
-   property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote
-   is cast.
+If the number of Nodes that have voted is equal to the number specified by the `nifi.cluster.flow.election.max.candidates`
+property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote
+is cast.
 ** `nifi.cluster.flow.election.max.candidates` - Specifies the number of Nodes required in the cluster to cause early election
-   of Flows. This allows the Nodes in the cluster to avoid having to wait a long time before starting processing if we reach
-   at least this number of nodes in the cluster.
+of Flows. This allows the Nodes in the cluster to avoid having to wait a long time before starting processing if we reach
+at least this number of nodes in the cluster.
 
 Now, it is possible to start up the cluster. It does not matter which order the instances start up. Navigate to the URL for
 one of the nodes, and the User Interface should look similar to the following:
@@ -2468,9 +2479,9 @@ Now, when we start NiFi, it will use Kerberos to authentication as the `nifi` us
 When using Kerberos, it is import to use fully-qualified domain names and not use _localhost_. Please ensure that the fully qualified hostname of each server is used
 in the following locations:
 
-	- _conf/zookeeper.properties_ file should use FQDN for `server.1`, `server.2`, ..., `server.N` values.
-	- The `Connect String` property of the ZooKeeperStateProvider
-	- The _/etc/hosts_ file should also resolve the FQDN to an IP address that is *not* `127.0.0.1`.
+- _conf/zookeeper.properties_ file should use FQDN for `server.1`, `server.2`, ..., `server.N` values.
+- The `Connect String` property of the ZooKeeperStateProvider
+- The _/etc/hosts_ file should also resolve the FQDN to an IP address that is *not* `127.0.0.1`.
 
 Failure to do so, may result in errors similar to the following:
 
@@ -2605,32 +2616,32 @@ take effect only after NiFi has been stopped and restarted.
 |*Property*|*Description*
 |`java`|Specifies the fully qualified java command to run. By default, it is simply `java` but could be changed to an absolute path or a reference an environment variable, such as `$JAVA_HOME/bin/java`
 |`run.as`|The username to run NiFi as. For instance, if NiFi should be run as the `nifi` user, setting this value to `nifi` will cause the NiFi Process to be run as the `nifi` user.
-        This property is ignored on Windows. For Linux, the specified user may require sudo permissions.
+This property is ignored on Windows. For Linux, the specified user may require sudo permissions.
 |`preserve.environment`|Whether or not to preserve shell environment while using `run.as` (see "sudo -E" man page). By default, this is set to false.
 |`lib.dir`|The _lib_ directory to use for NiFi. By default, this is set to `./lib`
 |`conf.dir`|The _conf_ directory to use for NiFi. By default, this is set to `./conf`
 |`graceful.shutdown.seconds`|When NiFi is instructed to shutdown, the Bootstrap will wait this number of seconds for the process to shutdown cleanly. At this amount of time,
-                           if the service is still running, the Bootstrap will `kill` the process, or terminate it abruptly.
+if the service is still running, the Bootstrap will `kill` the process, or terminate it abruptly.
 |`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, Java IO temporary directory, etc.
+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, Java IO temporary directory, etc.
 |`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 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.
+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.
 |`notification.max.attempts`|If a notification service is configured but is unable to perform its function, it will try again up to a maximum number of attempts. This property
-                           configures what that maximum number of attempts is. The default value is `5`.
+configures what that maximum number of attempts is. The default value is `5`.
 |`nifi.start.notification.services`|This property is a comma-separated list of Notification Service identifiers that correspond to the Notification Services
-                                  defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
-                                  configured recipients whenever NiFi is started.
+defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
+configured recipients whenever NiFi is started.
 |`nifi.stop.notification.services`|This property is a comma-separated list of Notification Service identifiers that correspond to the Notification Services
-                                 defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
-                                 configured recipients whenever NiFi is stopped.
+defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
+configured recipients whenever NiFi is stopped.
 |`nifi.died.notification.services`|This property is a comma-separated list of Notification Service identifiers that correspond to the Notification Services
-                                 defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
-                                 configured recipients if the bootstrap determines that NiFi has unexpectedly died.
+defined in the `notification.services.file` property. The services with the specified identifiers will be used to notify their
+configured recipients if the bootstrap determines that NiFi has unexpectedly died.
 |====
 
 [[notification_services]]
@@ -2678,7 +2689,7 @@ It has the following properties available:
 |`SMTP X-Mailer Header`||X-Mailer used in the header of the outgoing email
 |`Content Type`||Mime Type used to interpret the contents of the email, such as `text/plain` or `text/html`
 |`From`|true|Specifies the Email address to use as the sender. Otherwise, a "friendly name" can be used as the From address, but the value
-           must be enclosed in double-quotes.
+must be enclosed in double-quotes.
 |`To`||The recipients to include in the To-Line of the email
 |`CC`||The recipients to include in the CC-Line of the email
 |`BCC`||The recipients to include in the BCC-Line of the email
@@ -3165,14 +3176,14 @@ FlowFile Repository, if also on that disk, could become corrupt. To avoid this s
 |`nifi.content.repository.implementation`|The Content Repository implementation. The default value is `org.apache.nifi.controller.repository.FileSystemRepository` and should only be changed with caution. To store flowfile content in memory instead of on disk (at the risk of data loss in the event of power/machine failure), set this property to `org.apache.nifi.controller.repository.VolatileContentRepository`.
 |`nifi.content.claim.max.appendable.size`|The maximum size for a content claim. The default value is `1 MB`.
 |`nifi.content.repository.directory.default`*|The location of the Content Repository. The default value is `./content_repository`. +
- +
++
 *NOTE*: Multiple content repositories can be specified by using the `nifi.content.repository.directory.` prefix with unique suffixes and separate paths as values. +
- +
++
 For example, to provide two additional locations to act as part of the content repository, a user could also specify additional properties with keys of: +
- +
++
 `nifi.content.repository.directory.content1=/repos/content1` +
 `nifi.content.repository.directory.content2=/repos/content2` +
- +
++
 Providing three total locations, including  `nifi.content.repository.directory.default`.
 |`nifi.content.repository.archive.max.retention.period`|If archiving is enabled (see `nifi.content.repository.archive.enabled` below), then
 this property specifies the maximum amount of time to keep the archived data. The default value is `12 hours`.
@@ -3189,12 +3200,12 @@ All of the properties defined above (see <<file-system-content-repository-proper
 
 |====
 |*Property*|*Description*
- |`nifi.content.repository.encryption.key.provider.implementation`|This is the fully-qualified class name of the **key provider**. A key provider is the datastore interface for accessing the encryption key to protect the content claims. There are currently three implementations: `StaticKeyProvider` which reads a key directly from _nifi.properties_, `FileBasedKeyProvider` which reads keys from an encrypted file, and `KeyStoreKeyProvider` which reads keys from a standard `java.security.KeyStore`.
- |`nifi.content.repository.encryption.key.provider.location`|The path to the key definition resource (empty for `StaticKeyProvider`, `./keys.nkp` or similar path for `FileBasedKeyProvider`). For future providers like an HSM, this may be a connection string or URL.
- |`nifi.content.repository.encryption.key.provider.password`|The password used for decrypting the key definition resource, such as the keystore for `KeyStoreKeyProvider`.
- |`nifi.content.repository.encryption.key.id`|The active key ID to use for encryption (e.g. `Key1`).
- |`nifi.content.repository.encryption.key`|The key to use for `StaticKeyProvider`. The key format is hex-encoded (`0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210`) but can also be encrypted using the `./encrypt-config.sh` tool in NiFi Toolkit (see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Config Tool>> section in the link:toolkit-guide.html[NiFi Toolkit Guide] for more information).
- |`nifi.content.repository.encryption.key.id.`*|Allows for additional keys to be specified for the `StaticKeyProvider`. For example, the line `nifi.content.repository.encryption.key.id.Key2=012...210` would provide an available key `Key2`.
+|`nifi.content.repository.encryption.key.provider.implementation`|This is the fully-qualified class name of the **key provider**. A key provider is the datastore interface for accessing the encryption key to protect the content claims. There are currently three implementations: `StaticKeyProvider` which reads a key directly from _nifi.properties_, `FileBasedKeyProvider` which reads keys from an encrypted file, and `KeyStoreKeyProvider` which reads keys from a standard `java.security.KeyStore`.
+|`nifi.content.repository.encryption.key.provider.location`|The path to the key definition resource (empty for `StaticKeyProvider`, `./keys.nkp` or similar path for `FileBasedKeyProvider`). For future providers like an HSM, this may be a connection string or URL.
+|`nifi.content.repository.encryption.key.provider.password`|The password used for decrypting the key definition resource, such as the keystore for `KeyStoreKeyProvider`.
+|`nifi.content.repository.encryption.key.id`|The active key ID to use for encryption (e.g. `Key1`).
+|`nifi.content.repository.encryption.key`|The key to use for `StaticKeyProvider`. The key format is hex-encoded (`0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210`) but can also be encrypted using the `./encrypt-config.sh` tool in NiFi Toolkit (see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Config Tool>> section in the link:toolkit-guide.html[NiFi Toolkit Guide] for more information).
+|`nifi.content.repository.encryption.key.id.`*|Allows for additional keys to be specified for the `StaticKeyProvider`. For example, the line `nifi.content.repository.encryption.key.id.Key2=012...210` would provide an available key `Key2`.
 |====
 
 The simplest configuration is below:
@@ -3252,43 +3263,43 @@ the `WriteAheadProvenanceRepository`, it cannot be changed back to the `Persiste
 |====
 |*Property*|*Description*
 |`nifi.provenance.repository.directory.default`*|The location of the Provenance Repository. The default value is `./provenance_repository`. +
- +
-	*NOTE*: Multiple provenance repositories can be specified by using the `nifi.provenance.repository.directory.` prefix with unique suffixes and separate paths as values. +
- +
-	For example, to provide two additional locations to act as part of the provenance repository, a user could also specify additional properties with keys of: +
- +
-	`nifi.provenance.repository.directory.provenance1=/repos/provenance1` +
-	`nifi.provenance.repository.directory.provenance2=/repos/provenance2` +
- +
-	Providing three total locations, including `nifi.provenance.repository.directory.default`.
++
+*NOTE*: Multiple provenance repositories can be specified by using the `nifi.provenance.repository.directory.` prefix with unique suffixes and separate paths as values. +
++
+For example, to provide two additional locations to act as part of the provenance repository, a user could also specify additional properties with keys of: +
++
+`nifi.provenance.repository.directory.provenance1=/repos/provenance1` +
+`nifi.provenance.repository.directory.provenance2=/repos/provenance2` +
++
+Providing three total locations, including `nifi.provenance.repository.directory.default`.
 |`nifi.provenance.repository.max.storage.time`|The maximum amount of time to keep data provenance information. The default value is `24 hours`.
 |`nifi.provenance.repository.max.storage.size`|The maximum amount of data provenance information to store at a time.
-	The default value is `10 GB`. The Data Provenance capability can consume a great deal of storage space because so much data is kept.
-	For production environments, values of 1-2 TB or more is not uncommon. The repository will write to a single "event file" (or set of
-	"event files" if multiple storage locations are defined, as described above) until the event file reaches the size defined in the `nifi.provenance.repository.rollover.size` property. It will then "roll over" and begin writing new events to a new file. Data is always aged off one file at a time, so it is not advisable to write a tremendous amount of data to a single "event file," as it will prevent old data from aging off as smoothly.
+The default value is `10 GB`. The Data Provenance capability can consume a great deal of storage space because so much data is kept.
+For production environments, values of 1-2 TB or more is not uncommon. The repository will write to a single "event file" (or set of
+"event files" if multiple storage locations are defined, as described above) until the event file reaches the size defined in the `nifi.provenance.repository.rollover.size` property. It will then "roll over" and begin writing new events to a new file. Data is always aged off one file at a time, so it is not advisable to write a tremendous amount of data to a single "event file," as it will prevent old data from aging off as smoothly.
 |`nifi.provenance.repository.rollover.size`|The amount of data to write to a single "event file." The default value is `100 MB`. For production
-	environments where a very large amount of Data Provenance is generated, a value of `1 GB` is also very reasonable.
+environments where a very large amount of Data Provenance is generated, a value of `1 GB` is also very reasonable.
 |`nifi.provenance.repository.query.threads`|The number of threads to use for Provenance Repository queries. The default value is `2`.
 |`nifi.provenance.repository.index.threads`|The number of threads to use for indexing Provenance events so that they are searchable. The default value is `2`.
-	For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this happens, increasing the
-	value of this property may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput.
-	It is advisable to use at least 1 thread per storage location (i.e., if there are 3 storage locations, at least 3 threads should be used). For high
-	throughput environments, where more CPU and disk I/O is available, it may make sense to increase this value significantly. Typically going beyond
-	2-4 threads per storage location is not valuable. However, this can be tuned depending on the CPU resources available compared to the I/O resources.
+For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this happens, increasing the
+value of this property may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput.
+It is advisable to use at least 1 thread per storage location (i.e., if there are 3 storage locations, at least 3 threads should be used). For high
+throughput environments, where more CPU and disk I/O is available, it may make sense to increase this value significantly. Typically going beyond
+2-4 threads per storage location is not valuable. However, this can be tuned depending on the CPU resources available compared to the I/O resources.
 |`nifi.provenance.repository.compress.on.rollover`|Indicates whether to compress the provenance information when an "event file" is rolled over. The default value is `true`.
 |`nifi.provenance.repository.always.sync`|If set to `true`, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system
-	not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is `false`, there could be the potential for data
-	loss if either there is a sudden power loss or the operating system crashes. The default value is `false`.
+not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is `false`, there could be the potential for data
+loss if either there is a sudden power loss or the operating system crashes. The default value is `false`.
 |`nifi.provenance.repository.indexed.fields`|This is a comma-separated list of the fields that should be indexed and made searchable.
-	Fields that are not indexed will not be searchable. Valid fields are: `EventType`, `FlowFileUUID`, `Filename`, `TransitURI`, `ProcessorID`,
-	`AlternateIdentifierURI`, `Relationship`, `Details`. The default value is: `EventType, FlowFileUUID, Filename, ProcessorID`.
+Fields that are not indexed will not be searchable. Valid fields are: `EventType`, `FlowFileUUID`, `Filename`, `TransitURI`, `ProcessorID`,
+`AlternateIdentifierURI`, `Relationship`, `Details`. The default value is: `EventType, FlowFileUUID, Filename, ProcessorID`.
 |`nifi.provenance.repository.indexed.attributes`|This is a comma-separated list of FlowFile Attributes that should be indexed and made searchable. It is blank by default.
-	But some good examples to consider are `filename` and `mime.type` as well as any custom attributes you might use which are valuable for your use case.
+But some good examples to consider are `filename` and `mime.type` as well as any custom attributes you might use which are valuable for your use case.
 |`nifi.provenance.repository.index.shard.size`|The repository uses Apache Lucene to performing indexing and searching capabilities. This value indicates how large a Lucene Index should
-	become before the Repository starts writing to a new Index. Large values for the shard size will result in more Java heap usage when searching the Provenance Repository but should
-	provide better performance. The default value is `500 MB`. However, this is due to the fact that defaults are tuned for very small environments where most users begin to use NiFi.
-	For production environments, it is advisable to change this value to `4` to `8 GB`. Once all Provenance Events in the index have been aged off from the "event files," the index
-	will be destroyed as well.
+become before the Repository starts writing to a new Index. Large values for the shard size will result in more Java heap usage when searching the Provenance Repository but should
+provide better performance. The default value is `500 MB`. However, this is due to the fact that defaults are tuned for very small environments where most users begin to use NiFi.
+For production environments, it is advisable to change this value to `4` to `8 GB`. Once all Provenance Events in the index have been aged off from the "event files," the index
+will be destroyed as well.
 
   *NOTE:* This value should be smaller than (no more than half of) the `nifi.provenance.repository.max.storage.size` property.
 |`nifi.provenance.repository.max.attribute.length`|Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from the repository.
@@ -3316,12 +3327,12 @@ All of the properties defined above (see <<write-ahead-provenance-repository-pro
 
 |====
 |*Property*|*Description*
- |`nifi.provenance.repository.encryption.key.provider.implementation`|This is the fully-qualified class name of the **key provider**. A key provider is the datastore interface for accessing the encryption key to protect the provenance events. There are currently three implementations: `StaticKeyProvider` which reads a key directly from _nifi.properties_, `FileBasedKeyProvider` which reads keys from an encrypted file, and `KeyStoreKeyProvider` which reads keys from a standard `java.securi [...]
- |`nifi.provenance.repository.encryption.key.provider.location`|The path to the key definition resource (empty for `StaticKeyProvider`, `./keys.nkp` or similar path for `FileBasedKeyProvider`). For future providers like an HSM, this may be a connection string or URL.
- |`nifi.provenance.repository.encryption.key.provider.password`|The password used for decrypting the key definition resource, such as the keystore for `KeyStoreKeyProvider`.
- |`nifi.provenance.repository.encryption.key.id`|The active key ID to use for encryption (e.g. `Key1`).
- |`nifi.provenance.repository.encryption.key`|The key to use for `StaticKeyProvider`. The key format is hex-encoded (`0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210`) but can also be encrypted using the `./encrypt-config.sh` tool in NiFi Toolkit (see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Config Tool>> section in the link:toolkit-guide.html[NiFi Toolkit Guide] for more information).
- |`nifi.provenance.repository.encryption.key.id.`*|Allows for additional keys to be specified for the `StaticKeyProvider`. For example, the line `nifi.provenance.repository.encryption.key.id.Key2=012...210` would provide an available key `Key2`.
+|`nifi.provenance.repository.encryption.key.provider.implementation`|This is the fully-qualified class name of the **key provider**. A key provider is the datastore interface for accessing the encryption key to protect the provenance events. There are currently three implementations: `StaticKeyProvider` which reads a key directly from _nifi.properties_, `FileBasedKeyProvider` which reads keys from an encrypted file, and `KeyStoreKeyProvider` which reads keys from a standard `java.securit [...]
+|`nifi.provenance.repository.encryption.key.provider.location`|The path to the key definition resource (empty for `StaticKeyProvider`, `./keys.nkp` or similar path for `FileBasedKeyProvider`). For future providers like an HSM, this may be a connection string or URL.
+|`nifi.provenance.repository.encryption.key.provider.password`|The password used for decrypting the key definition resource, such as the keystore for `KeyStoreKeyProvider`.
+|`nifi.provenance.repository.encryption.key.id`|The active key ID to use for encryption (e.g. `Key1`).
+|`nifi.provenance.repository.encryption.key`|The key to use for `StaticKeyProvider`. The key format is hex-encoded (`0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210`) but can also be encrypted using the `./encrypt-config.sh` tool in NiFi Toolkit (see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Config Tool>> section in the link:toolkit-guide.html[NiFi Toolkit Guide] for more information).
+|`nifi.provenance.repository.encryption.key.id.`*|Allows for additional keys to be specified for the `StaticKeyProvider`. For example, the line `nifi.provenance.repository.encryption.key.id.Key2=012...210` would provide an available key `Key2`.
 |====
 
 The simplest configuration is below:
@@ -3340,14 +3351,14 @@ nifi.provenance.repository.encryption.key=0123456789ABCDEFFEDCBA9876543210012345
 |====
 |*Property*|*Description*
 |`nifi.provenance.repository.directory.default`*|The location of the Provenance Repository. The default value is `./provenance_repository`. +
- +
++
 *NOTE*: Multiple provenance repositories can be specified by using the `nifi.provenance.repository.directory.` prefix with unique suffixes and separate paths as values. +
- +
++
 For example, to provide two additional locations to act as part of the provenance repository, a user could also specify additional properties with keys of: +
- +
++
 `nifi.provenance.repository.directory.provenance1=/repos/provenance1` +
 `nifi.provenance.repository.directory.provenance2=/repos/provenance2` +
- +
++
 Providing three total locations, including `nifi.provenance.repository.directory.default`.
 |`nifi.provenance.repository.max.storage.time`|The maximum amount of time to keep data provenance information. The default value is `24 hours`.
 |`nifi.provenance.repository.max.storage.size`|The maximum amount of data provenance information to store at a time. The default value is `10 GB`.
@@ -3355,9 +3366,9 @@ Providing three total locations, including `nifi.provenance.repository.directory
 |`nifi.provenance.repository.rollover.size`|The amount of information to roll over at a time. The default value is `100 MB`.
 |`nifi.provenance.repository.query.threads`|The number of threads to use for Provenance Repository queries. The default value is `2`.
 |`nifi.provenance.repository.index.threads`|The number of threads to use for indexing Provenance events so that they are searchable. The default value is `2`.
-	For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this is the case, a bulletin will appear, indicating that
-	"The rate of the dataflow is exceeding the provenance recording rate. Slowing down flow to accommodate." If this happens, increasing the value of this property
-	may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput.
+For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this is the case, a bulletin will appear, indicating that
+"The rate of the dataflow is exceeding the provenance recording rate. Slowing down flow to accommodate." If this happens, increasing the value of this property
+may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput.
 |`nifi.provenance.repository.compress.on.rollover`|Indicates whether to compress the provenance information when rolling it over. The default value is `true`.
 |`nifi.provenance.repository.always.sync`|If set to `true`, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is `false`, there could be the potential for data loss if either there is a sudden power loss or the operating system crashes. The default value is `false`.
 |`nifi.provenance.repository.journal.count`|The number of journal files that should be used to serialize Provenance Event data. Increasing this value will allow more tasks to simultaneously update the repository but will result in more expensive merging of the journal files later. This value should ideally be equal to the number of threads that are expected to update the repository simultaneously, but 16 tends to work well in must environments. The default value is `16`.
@@ -3700,14 +3711,14 @@ These properties pertain to the web-based User Interface.
 |`nifi.web.http.port`|The HTTP port. The default value is blank.
 |`nifi.web.http.port.forwarding`|The port which forwards incoming HTTP requests to `nifi.web.http.host`. This property is designed to be used with 'port forwarding', when NiFi has to be started by a non-root user for better security, yet it needs to be accessed via low port to go through a firewall. For example, to expose NiFi via HTTP protocol on port 80, but actually listening on port 8080, you need to configure OS level port forwarding such as `iptables` (Linux/Unix) or `pfctl` (macOS [...]
 |`nifi.web.http.network.interface`*|The name of the network interface to which NiFi should bind for HTTP requests. It is blank by default. +
- +
++
 *NOTE*: Multiple network interfaces can be specified by using the `nifi.web.http.network.interface.` prefix with unique suffixes and separate network interface names as values. +
- +
++
 For example, to provide two additional network interfaces, a user could also specify additional properties with keys of: +
- +
++
 `nifi.web.http.network.interface.eth0=eth0` +
 `nifi.web.http.network.interface.eth1=eth1` +
- +
++
 Providing three total network interfaces, including  `nifi.web.http.network.interface.default`.
 |`nifi.web.https.host`|The HTTPS host. The default value is `127.0.0.1`.
 |`nifi.web.https.port`|The HTTPS port. The default value is `8443`.
@@ -3721,14 +3732,14 @@ In Firefox, the SSL cipher negotiated with Jetty may be examined in the 'Secure
 
 
 |`nifi.web.https.network.interface`*|The name of the network interface to which NiFi should bind for HTTPS requests. It is blank by default. +
- +
++
 *NOTE*: Multiple network interfaces can be specified by using the `nifi.web.https.network.interface.` prefix with unique suffixes and separate network interface names as values. +
- +
++
 For example, to provide two additional network interfaces, a user could also specify additional properties with keys of: +
- +
++
 `nifi.web.https.network.interface.eth0=eth0` +
 `nifi.web.https.network.interface.eth1=eth1` +
- +
++
 Providing three total network interfaces, including  `nifi.web.https.network.interface.default`.
 |`nifi.web.jetty.working.directory`|The location of the Jetty working directory. The default value is `./work/jetty`.
 |`nifi.web.jetty.threads`|The number of Jetty threads. The default value is `200`.
@@ -3840,7 +3851,7 @@ from the remote node before considering the communication with the node a failur
 |`nifi.cluster.firewall.file`|The location of the node firewall file. This is a file that may be used to list all the nodes that are allowed to connect
 to the cluster. It provides an additional layer of security. This value is blank by default, meaning that no firewall file is to be used. See <<cluster_firewall_configuration>> for file format details.
 |`nifi.cluster.flow.election.max.wait.time`|Specifies the amount of time to wait before electing a Flow as the "correct" Flow. If the number of Nodes that have voted is equal to the number specified
- by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast.
+by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast.
 |`nifi.cluster.flow.election.max.candidates`|Specifies the number of Nodes required in the cluster to cause early election of Flows. This allows the Nodes in the cluster to avoid having to wait a
 long time before starting processing if we reach at least this number of nodes in the cluster.
 |`nifi.cluster.load.balance.port`|Specifies the port to listen on for incoming connections for load balancing data across the cluster. The default value is `6342`.
@@ -3885,17 +3896,17 @@ that is specified.
 |====
 |*Property*|*Description*
 |`nifi.kerberos.krb5.file`*|The location of the krb5 file, if used. It is blank by default. At this time, only a single krb5 file is allowed to
-    be specified per NiFi instance, so this property is configured here to support SPNEGO and service principals rather than in individual Processors.
-    If necessary the krb5 file can support multiple realms.
-    Example: `/etc/krb5.conf`
+be specified per NiFi instance, so this property is configured here to support SPNEGO and service principals rather than in individual Processors.
+If necessary the krb5 file can support multiple realms.
+Example: `/etc/krb5.conf`
 |`nifi.kerberos.service.principal`*|The name of the NiFi Kerberos service principal, if used. It is blank by default. Note that this property is for NiFi to authenticate as a client other systems.
-   Example: `nifi/nifi.example.com` or `nifi/nifi.example.com@EXAMPLE.COM`
+Example: `nifi/nifi.example.com` or `nifi/nifi.example.com@EXAMPLE.COM`
 |`nifi.kerberos.service.keytab.location`*|The file path of the NiFi Kerberos keytab, if used. It is blank by default. Note that this property is for NiFi to authenticate as a client other systems.
-  Example: `/etc/nifi.keytab`
+Example: `/etc/nifi.keytab`
 |`nifi.kerberos.spnego.principal`*|The name of the NiFi Kerberos service principal, if used. It is blank by default. Note that this property is used to authenticate NiFi users.
-   Example: `HTTP/nifi.example.com` or `HTTP/nifi.example.com@EXAMPLE.COM`
+Example: `HTTP/nifi.example.com` or `HTTP/nifi.example.com@EXAMPLE.COM`
 |`nifi.kerberos.spnego.keytab.location`*|The file path of the NiFi Kerberos keytab, if used. It is blank by default. Note that this property is used to authenticate NiFi users.
-  Example: `/etc/http-nifi.keytab`
+Example: `/etc/http-nifi.keytab`
 |`nifi.kerberos.spengo.authentication.expiration`*|The expiration duration of a successful Kerberos user authentication, if used. The default value is `12 hours`.
 |====
 
@@ -4117,12 +4128,12 @@ $ ./nifi-toolkit-1.10.0/bin/encrypt-config.sh -f /path/to/nifi/nifi-1.9.2/conf/f
 
 where:
 
-    * `-f` specifies the source _flow.xml.gz_ (nifi-1.9.2)
-    * `-g` specifies the destination _flow.xml.gz_ (nifi-1.10.0)
-    * `-s` specifies the new sensitive properties key (`new_password`)
-    * `-n` specifies the source _nifi.properties_ (nifi-1.9.2)
-    * `-o` specifies the destination _nifi.properties_ (nifi-1.10.0)
-    * `-x` tells Encrypt-Config to only process the sensitive properties
+* `-f` specifies the source _flow.xml.gz_ (nifi-1.9.2)
+* `-g` specifies the destination _flow.xml.gz_ (nifi-1.10.0)
+* `-s` specifies the new sensitive properties key (`new_password`)
+* `-n` specifies the source _nifi.properties_ (nifi-1.9.2)
+* `-o` specifies the destination _nifi.properties_ (nifi-1.10.0)
+* `-x` tells Encrypt-Config to only process the sensitive properties
 
 For more information see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Config Tool>> section in the NiFi Toolkit Guide.
 
@@ -4144,8 +4155,8 @@ In your new NiFi installation:
 
 1. Start each of your new NiFi instances.
 2. Verify that:
-   * All your dataflows have returned to a running state. Some processors may have new properties that need to be configured, in which case they will be stopped and marked Invalid (image:iconAlert.png["Invalid"]).
-   * All your expected controller services and reporting tasks are running again. Address any controller services or reporting tasks that are marked Invalid (image:iconAlert.png["Invalid"]).
+* All your dataflows have returned to a running state. Some processors may have new properties that need to be configured, in which case they will be stopped and marked Invalid (image:iconAlert.png["Invalid"]).
+* All your expected controller services and reporting tasks are running again. Address any controller services or reporting tasks that are marked Invalid (image:iconAlert.png["Invalid"]).
 3. After confirming your new NiFi instances are stable and working as expected, the old installation can be removed.
 
 NOTE:  If the original NiFi was setup to run as a service, update any symlinks or service scripts to point to the new NiFi version executables.
diff --git a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
index 3b173a8..1738fa3 100644
--- a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
@@ -424,66 +424,69 @@ To show help:
 ==== NiFi
 The following are available options when targeting NiFi:
 
- * `-h`,`--help`                                 Show usage information (this message)
- * `-v`,`--verbose`                              Sets verbose mode (default false)
- * `-n`,`--niFiProperties <file>`                The _nifi.properties_ file containing unprotected config values (will be overwritten unless `-o` is specified)
- * `-o`,`--outputNiFiProperties <file>`          The destination _nifi.properties_ file containing protected config values (will not modify input _nifi.properties_)
- * `-l`,`--loginIdentityProviders <file>`        The _login-identity-providers.xml_ file containing unprotected config values (will be overwritten unless `-i` is specified)
- * `-i`,`--outputLoginIdentityProviders <file>`  The destination _login-identity-providers.xml_ file containing protected config values (will not modify input _login-identity-providers.xml_)
- * `-a`,`--authorizers <file>`                   The _authorizers.xml_ file containing unprotected config values (will be overwritten unless `-u` is specified)
- * `-u`,`--outputAuthorizers <file>`             The destination _authorizers.xml_ file containing protected config values (will not modify input _authorizers.xml_)
- * `-f`,`--flowXml <file>`                       The _flow.xml.gz_ file currently protected with old password (will be overwritten unless `-g` is specified)
- * `-g`,`--outputFlowXml <file>`                 The destination _flow.xml.gz_ file containing protected config values (will not modify input _flow.xml.gz_)
- * `-b`,`--bootstrapConf <file>`                 The bootstrap.conf file to persist root key and to optionally provide any configuration for the protection scheme.
- * `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [AES_GCM, HASHICORP_VAULT_TRANSIT, AWS_KMS] (default is AES_GCM)
- * `-k`,`--key <keyhex>`                         The raw hexadecimal key to use to encrypt the sensitive properties
- * `-e`,`--oldKey <keyhex>`                      The old raw hexadecimal key to use during key migration
- * `-H`,`--oldProtectionScheme <protectionScheme>` The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM
- * `-p`,`--password <password>`                  The password from which to derive the key to use to encrypt the sensitive properties
- * `-w`,`--oldPassword <password>`            The old password from which to derive the key during migration
- * `-r`,`--useRawKey`                            If provided, the secure console will prompt for the raw key value in hexadecimal form
- * `-m`,`--migrate`                              If provided, the _nifi.properties_ and/or _login-identity-providers.xml_ sensitive properties will be re-encrypted with the new scheme
- * `-x`,`--encryptFlowXmlOnly`                   If provided, the properties in _flow.xml.gz_ will be re-encrypted with a new key but the _nifi.properties_ and/or _login-identity-providers.xml_ files will not be modified
- * `-s`,`--propsKey <password|keyhex>`           The password or key to use to encrypt the sensitive processor properties in _flow.xml.gz_
- * `-A`,`--newFlowAlgorithm <algorithm>`         The algorithm to use to encrypt the sensitive processor properties in _flow.xml.gz_
- * `-P`,`--newFlowProvider <algorithm>`          The security provider to use to encrypt the sensitive processor properties in _flow.xml.gz_
- * `-c`,`--translateCli`                         Translates the _nifi.properties_ file to a format suitable for the NiFi CLI tool
+* `-h`,`--help`                                 Show usage information (this message)
+* `-v`,`--verbose`                              Sets verbose mode (default false)
+* `-n`,`--niFiProperties <file>`                The _nifi.properties_ file containing unprotected config values (will be overwritten unless `-o` is specified)
+* `-o`,`--outputNiFiProperties <file>`          The destination _nifi.properties_ file containing protected config values (will not modify input _nifi.properties_)
+* `-l`,`--loginIdentityProviders <file>`        The _login-identity-providers.xml_ file containing unprotected config values (will be overwritten unless `-i` is specified)
+* `-i`,`--outputLoginIdentityProviders <file>`  The destination _login-identity-providers.xml_ file containing protected config values (will not modify input _login-identity-providers.xml_)
+* `-a`,`--authorizers <file>`                   The _authorizers.xml_ file containing unprotected config values (will be overwritten unless `-u` is specified)
+* `-u`,`--outputAuthorizers <file>`             The destination _authorizers.xml_ file containing protected config values (will not modify input _authorizers.xml_)
+* `-f`,`--flowXml <file>`                       The _flow.xml.gz_ file currently protected with old password (will be overwritten unless `-g` is specified)
+* `-g`,`--outputFlowXml <file>`                 The destination _flow.xml.gz_ file containing protected config values (will not modify input _flow.xml.gz_)
+* `-b`,`--bootstrapConf <file>`                 The bootstrap.conf file to persist root key and to optionally provide any configuration for the protection scheme.
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>] (default is AES_GCM)
+* `-k`,`--key <keyhex>`                         The raw hexadecimal key to use to encrypt the sensitive properties
+* `-e`,`--oldKey <keyhex>`                      The old raw hexadecimal key to use during key migration
+* `-H`,`--oldProtectionScheme <protectionScheme>` The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM
+* `-p`,`--password <password>`                  The password from which to derive the key to use to encrypt the sensitive properties
+* `-w`,`--oldPassword <password>`            The old password from which to derive the key during migration
+* `-r`,`--useRawKey`                            If provided, the secure console will prompt for the raw key value in hexadecimal form
+* `-m`,`--migrate`                              If provided, the _nifi.properties_ and/or _login-identity-providers.xml_ sensitive properties will be re-encrypted with the new scheme
+* `-x`,`--encryptFlowXmlOnly`                   If provided, the properties in _flow.xml.gz_ will be re-encrypted with a new key but the _nifi.properties_ and/or _login-identity-providers.xml_ files will not be modified
+* `-s`,`--propsKey <password|keyhex>`           The password or key to use to encrypt the sensitive processor properties in _flow.xml.gz_
+* `-A`,`--newFlowAlgorithm <algorithm>`         The algorithm to use to encrypt the sensitive processor properties in _flow.xml.gz_
+* `-P`,`--newFlowProvider <algorithm>`          The security provider to use to encrypt the sensitive processor properties in _flow.xml.gz_
+* `-c`,`--translateCli`                         Translates the _nifi.properties_ file to a format suitable for the NiFi CLI tool
 
 ==== NiFi Registry
 The following are available options when targeting NiFi Registry using the `--nifiRegistry` flag:
 
- * `-h`,`--help`                                 Show usage information (this message)
- * `-v`,`--verbose`                              Sets verbose mode (default false)
- * `-p`,`--password <password>`                  Protect the files using a password-derived key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the password.
- * `-k`,`--key <keyhex>`                         Protect the files using a raw hexadecimal key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the key.
- * `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [AES_GCM, HASHICORP_VAULT_TRANSIT, AWS_KMS]  (default is AES_GCM)
- * `--oldPassword <password>`                    If the input files are already protected using a password-derived key, this specifies the old password so that the files can be unprotected before re-protecting.
- * `--oldKey <keyhex>`                           If the input files are already protected using a key, this specifies the raw hexadecimal key so that the files can be unprotected before re-protecting.
- * `-H`,`--oldProtectionScheme <protectionScheme>`The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM.
- * `-b`,`--bootstrapConf <file>`                 The _bootstrap.conf_ file containing no root key or an existing root key, and any other protection scheme configuration properties. 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.
- * `-B`,`--outputBootstrapConf <file>`           The destination _bootstrap.conf_ file to persist root key. If specified, the input _bootstrap.conf_ will not be modified.
- * `-r`,`--nifiRegistryProperties <file>`        The _nifi-registry.properties_ file containing unprotected config values, overwritten if no output file specified.
- * `-R`,`--outputNifiRegistryProperties <file>`  The destination _nifi-registry.properties_ file containing protected config values.
- * `-a`,`--authorizersXml <file>`                The _authorizers.xml_ file containing unprotected config values, overwritten if no output file specified.
- * `-A`,`--outputAuthorizersXml <file>`          The destination _authorizers.xml_ file containing protected config values.
- * `-i`,`--identityProvidersXml <file>`          The _identity-providers.xml_ file containing unprotected config values, overwritten if no output file specified.
- * `-I`,`--outputIdentityProvidersXml <file>`    The destination _identity-providers.xml_ file containing protected config values.
- * `--decrypt`                                    Can be used with `-r` to decrypt a previously encrypted NiFi Registry Properties file. Decrypted content is printed to STDOUT.
+* `-h`,`--help`                                 Show usage information (this message)
+* `-v`,`--verbose`                              Sets verbose mode (default false)
+* `-p`,`--password <password>`                  Protect the files using a password-derived key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the password.
+* `-k`,`--key <keyhex>`                         Protect the files using a raw hexadecimal key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the key.
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>]  (default is AES_GCM)
+* `--oldPassword <password>`                    If the input files are already protected using a password-derived key, this specifies the old password so that the files can be unprotected before re-protecting.
+* `--oldKey <keyhex>`                           If the input files are already protected using a key, this specifies the raw hexadecimal key so that the files can be unprotected before re-protecting.
+* `-H`,`--oldProtectionScheme <protectionScheme>`The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM.
+* `-b`,`--bootstrapConf <file>`                 The _bootstrap.conf_ file containing no root key or an existing root key, and any other protection scheme configuration properties. 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.
+* `-B`,`--outputBootstrapConf <file>`           The destination _bootstrap.conf_ file to persist root key. If specified, the input _bootstrap.conf_ will not be modified.
+* `-r`,`--nifiRegistryProperties <file>`        The _nifi-registry.properties_ file containing unprotected config values, overwritten if no output file specified.
+* `-R`,`--outputNifiRegistryProperties <file>`  The destination _nifi-registry.properties_ file containing protected config values.
+* `-a`,`--authorizersXml <file>`                The _authorizers.xml_ file containing unprotected config values, overwritten if no output file specified.
+* `-A`,`--outputAuthorizersXml <file>`          The destination _authorizers.xml_ file containing protected config values.
+* `-i`,`--identityProvidersXml <file>`          The _identity-providers.xml_ file containing unprotected config values, overwritten if no output file specified.
+* `-I`,`--outputIdentityProvidersXml <file>`    The destination _identity-providers.xml_ file containing protected config values.
+* `--decrypt`                                    Can be used with `-r` to decrypt a previously encrypted NiFi Registry Properties file. Decrypted content is printed to STDOUT.
 
 === Protection Schemes
 The protection scheme can be selected during encryption using the `--protectionScheme` flag.  During migration, the former protection scheme is specified using the `--oldProtectionScheme` flag.  This distinction allows a set of protected configuration files to be migrated not only to a new key, but to a completely different protection scheme.
 
-==== AES_GCM
+==== AES_GCM [[AES_GCM]]
 The default protection scheme, `AES-G/CM` simply encrypts sensitive properties and marks their protection as either `aes/gcm/256` or `aes/gcm/256` as appropriate.  This protection is all done within NiFi itself.
 
-==== HASHICORP_VAULT_TRANSIT
-This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://www.vaultproject.io/docs/secrets/transit) to outsource encryption to a configured Vault server. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_TRANSIT protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `b [...]
+==== HASHICORP_VAULT_TRANSIT [[HASHICORP_VAULT_TRANSIT]]
+This protection scheme uses https://www.vaultproject.io/docs/secrets/transit[HashiCorp Vault Transit Secrets Engine] to outsource encryption to a configured Vault server. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_TRANSIT protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `boot [...]
 
-==== HASHICORP_VAULT_KV
-This protection scheme uses HashiCorp Vault's Transit unversioned Key/Value Engine (https://www.vaultproject.io/docs/secrets/kv/kv-v1) to store sensitive values as Vault Secrets. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_KV protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `b [...]
+==== HASHICORP_VAULT_KV [[HASHICORP_VAULT_KV]]
+This protection scheme uses https://www.vaultproject.io/docs/secrets/kv/kv-v1[HashiCorp Vault Key Value Secrets Engine Version 1] to store sensitive values as Vault Secrets. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_KV protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `bootst [...]
 
-==== AWS_KMS
-This protection scheme uses AWS Key Management Service (https://aws.amazon.com/kms/) for encryption and decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.  Therefore, w [...]
+==== AWS_KMS [[AWS_KMS]]
+This protection scheme uses https://aws.amazon.com/kms/[AWS Key Management] Service for encryption and decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.  Therefore, wh [...]
+
+==== AZURE_KEYVAULT_KEY [[AZURE_KEYVAULT_KEY]]
+This protection scheme uses keys managed by https://docs.microsoft.com/en-us/azure/key-vault/keys/about-keys[Azure Key Vault Keys] for encryption and decryption. Azure Key Vault configuration properties can be stored in the `bootstrap-azure.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. The provider will utilize the Azure default credentials provider chain as described in the https://docs.microsoft.com/en-us/java/api/overview/azure/security-keyvault-keys-read [...]
 
 === Examples
 
@@ -1102,13 +1105,13 @@ For a client certificate that can be easily imported into the browser, specify:
 In some enterprise scenarios, a security/IT team may provide a signing certificate that has already been signed by the organization's certificate authority (CA). This *intermediate CA* can be used to sign the *node* (sometimes referred to as *leaf*) certificates that will be installed on each NiFi node, or the *client certificates* used to identify users. In order to inject the existing signing certificate into the toolkit process, follow these steps:
 
 . Generate or obtain the signed intermediate CA keys in the following format (see additional commands below):
-  * Public certificate in PEM format: `nifi-cert.pem`
-  * Private key in PEM format: `nifi-key.key`
+* Public certificate in PEM format: `nifi-cert.pem`
+* Private key in PEM format: `nifi-key.key`
 . Place the files in the *toolkit working directory*. This is the directory where the tool is configured to output the signed certificates. *This is not necessarily the directory where the binary is located or invoked*.
-  * For example, given the following scenario, the toolkit command can be run from its location as long as the output directory `-o` is `../hardcoded/`, and the existing `nifi-cert.pem` and `nifi-key.key` will be used.
-  ** e.g. `$ ./toolkit/bin/tls-toolkit.sh standalone -o ./hardcoded/ -n 'node4.nifi.apache.org' -P thisIsABadPassword -S thisIsABadPassword -O` will result in a new directory at `./hardcoded/node4.nifi.apache.org` with a keystore and truststore containing a certificate signed by `./hardcoded/nifi-key.key`
-  * If the `-o` argument is not provided, the default working directory (`.`) must contain `nifi-cert.pem` and `nifi-key.key`
-  ** e.g. `$ cd ./hardcoded/ && ../toolkit/bin/tls-toolkit.sh standalone -n 'node5.nifi.apache.org' -P thisIsABadPassword -S thisIsABadPassword -O`
+* For example, given the following scenario, the toolkit command can be run from its location as long as the output directory `-o` is `../hardcoded/`, and the existing `nifi-cert.pem` and `nifi-key.key` will be used.
+** e.g. `$ ./toolkit/bin/tls-toolkit.sh standalone -o ./hardcoded/ -n 'node4.nifi.apache.org' -P thisIsABadPassword -S thisIsABadPassword -O` will result in a new directory at `./hardcoded/node4.nifi.apache.org` with a keystore and truststore containing a certificate signed by `./hardcoded/nifi-key.key`
+* If the `-o` argument is not provided, the default working directory (`.`) must contain `nifi-cert.pem` and `nifi-key.key`
+** e.g. `$ cd ./hardcoded/ && ../toolkit/bin/tls-toolkit.sh standalone -n 'node5.nifi.apache.org' -P thisIsABadPassword -S thisIsABadPassword -O`
 
 [source]
 ----
@@ -1410,26 +1413,26 @@ $ ./bin/tls-toolkit.sh standalone -n 'node1.nifi.apache.org' \
 === Additional Certificate Commands
 
 . To convert from DER encoded public certificate (`cert.der`) to PEM encoded (`cert.pem`):
-  * If the DER file contains both the public certificate and private key, remove the private key with this command:
-  ** `perl -pe 'BEGIN{undef $/;} s|-----BEGIN PRIVATE KEY-----.*?-----END PRIVATE KEY-----|Removed private key|gs' cert.der > cert.pem`
-  * If the DER file only contains the public certificate, use this command:
-  ** `openssl x509 -inform der -in cert.der -out cert.pem`
+* If the DER file contains both the public certificate and private key, remove the private key with this command:
+** `perl -pe 'BEGIN{undef $/;} s|-----BEGIN PRIVATE KEY-----.*?-----END PRIVATE KEY-----|Removed private key|gs' cert.der > cert.pem`
+* If the DER file only contains the public certificate, use this command:
+** `openssl x509 -inform der -in cert.der -out cert.pem`
 . To convert from a PKCS12 keystore (`keystore.p12`) containing both the public certificate and private key into PEM encoded files (`$PASSWORD` is the keystore password):
-  * `openssl pkcs12 -in keystore.p12 -nodes -clcerts -nokeys -out cert.pem  -password "pass:$PASSWORD"`
-  * `openssl pkcs12 -in keystore.p12 -nodes -nocerts -out key.key -password "pass:$PASSWORD"`
+* `openssl pkcs12 -in keystore.p12 -nodes -clcerts -nokeys -out cert.pem  -password "pass:$PASSWORD"`
+* `openssl pkcs12 -in keystore.p12 -nodes -nocerts -out key.key -password "pass:$PASSWORD"`
 . To convert from a Java Keystore (`keystore.jks`) containing private key into PEM encoded files (`$P12_PASSWORD` is the PKCS12 keystore password, `$JKS_PASSWORD` is the Java keystore password you want to set, and `$ALIAS` can be any value -- the NiFi default is `nifi-key`):
-  * `keytool -importkeystore -srckeystore keystore.jks -destkeystore keystore.p12 -srcstoretype JKS -deststoretype PKCS12 -destkeypass "$P12_PASSWORD" -deststorepass "$P12_PASSWORD" -srcstorepass "$JKS_PASSWORD" -srcalias "$ALIAS" -destalias "$ALIAS"`
-  * Follow the steps above to convert from `keystore.p12` to `cert.pem` and `key.key`
+* `keytool -importkeystore -srckeystore keystore.jks -destkeystore keystore.p12 -srcstoretype JKS -deststoretype PKCS12 -destkeypass "$P12_PASSWORD" -deststorepass "$P12_PASSWORD" -srcstorepass "$JKS_PASSWORD" -srcalias "$ALIAS" -destalias "$ALIAS"`
+* Follow the steps above to convert from `keystore.p12` to `cert.pem` and `key.key`
 . To convert from PKCS #8 PEM format to PKCS #1 PEM format:
-  * If the private key is provided in PKCS #8 format (the file begins with `-----BEGIN PRIVATE KEY-----` rather than `-----BEGIN RSA PRIVATE KEY-----`), the following command will convert it to PKCS #1 format, move the original to `nifi-key-pkcs8.key`, and rename the PKCS #1 version as `nifi-key.key`:
-  ** `openssl rsa -in nifi-key.key -out nifi-key-pkcs1.key && mv nifi-key.key nifi-key-pkcs8.key && mv nifi-key-pkcs1.key nifi-key.key`
+* If the private key is provided in PKCS #8 format (the file begins with `-----BEGIN PRIVATE KEY-----` rather than `-----BEGIN RSA PRIVATE KEY-----`), the following command will convert it to PKCS #1 format, move the original to `nifi-key-pkcs8.key`, and rename the PKCS #1 version as `nifi-key.key`:
+** `openssl rsa -in nifi-key.key -out nifi-key-pkcs1.key && mv nifi-key.key nifi-key-pkcs8.key && mv nifi-key-pkcs1.key nifi-key.key`
 . To combine a private key in PEM format (`private.key`) and public certificate in PEM format (`certificate.pem`) into PKCS12 keystore:
-  * The following command will create the PKCS12 keystore (`keystore.p12`) from the two independent files. A Java keystore (JKS) cannot be formed directly from the PEM files:
-  ** `openssl pkcs12 -export -out keystore.p12 -inkey private.key -in certificate.pem`
+* The following command will create the PKCS12 keystore (`keystore.p12`) from the two independent files. A Java keystore (JKS) cannot be formed directly from the PEM files:
+** `openssl pkcs12 -export -out keystore.p12 -inkey private.key -in certificate.pem`
 . To convert a PKCS12 keystore (`keystore.p12`) to JKS keystore (`keystore.jks`):
-  * The following command will create the JKS keystore (`keystore.jks`). The `-destalias` flag is optional, as NiFi does not currently read from a specific alias in the keystore. The user will be prompted for a keystore password, which must be set and have minimum 8 characters, and a key password, which can be the same as the keystore password or different:
-  ** `keytool -importkeystore -srckeystore keystore.p12 -srcstoretype pkcs12 -destkeystore keystore.jks
-       -deststoretype jks -destalias nifi-key`
+* The following command will create the JKS keystore (`keystore.jks`). The `-destalias` flag is optional, as NiFi does not currently read from a specific alias in the keystore. The user will be prompted for a keystore password, which must be set and have minimum 8 characters, and a key password, which can be the same as the keystore password or different:
+** `keytool -importkeystore -srckeystore keystore.p12 -srcstoretype pkcs12 -destkeystore keystore.jks
+-deststoretype jks -destalias nifi-key`
 
 [[zookeeper_migrator]]
 == ZooKeeper Migrator
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-azure.conf b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-azure.conf
new file mode 100644
index 0000000..f7cc47e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-azure.conf
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Azure KeyVault Key ID and Encryption Algorithm is required to be configured for Azure KeyVault Sensitive Property Provider
+azure.keyvault.key.id=
+azure.keyvault.encryption.algorithm=
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
index 4256a4b..4f348c2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
@@ -66,6 +66,9 @@ nifi.bootstrap.protection.hashicorp.vault.conf=./conf/bootstrap-hashicorp-vault.
 # AWS KMS Sensitive Property Providers
 nifi.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
 
+# Azure Key Vault Sensitive Property Providers
+nifi.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
+
 # Sets the provider of SecureRandom to /dev/urandom to prevent blocking on VMs
 java.arg.15=-Djava.security.egd=file:/dev/urandom
 
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-azure.conf b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-azure.conf
new file mode 100644
index 0000000..f7cc47e
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-azure.conf
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Azure KeyVault Key ID and Encryption Algorithm is required to be configured for Azure KeyVault Sensitive Property Provider
+azure.keyvault.key.id=
+azure.keyvault.encryption.algorithm=
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
index 046d252..2bcbc2e 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
@@ -60,3 +60,6 @@ nifi.registry.bootstrap.protection.hashicorp.vault.conf=./conf/bootstrap-hashico
 
 # AWS KMS Sensitive Property Providers
 nifi.registry.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
+
+# Azure Key Vault Sensitive Property Providers
+nifi.registry.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
\ No newline at end of file