You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by th...@apache.org on 2021/10/21 20:47:22 UTC

[nifi] branch main updated: NIFI-9263: Using secret-per-context model for Stateless Vault PVP

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

thenatog 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 207da40  NIFI-9263: Using secret-per-context model for Stateless Vault PVP
207da40 is described below

commit 207da405eeb47ea5848c0951b3108c7742752d25
Author: Joe Gresock <jg...@gmail.com>
AuthorDate: Fri Oct 1 10:29:37 2021 -0400

    NIFI-9263: Using secret-per-context model for Stateless Vault PVP
    
    Rebasing after PVP refactor
    
    Signed-off-by: Nathan Gough <th...@gmail.com>
    
    This closes #5427.
---
 .../HashiCorpVaultCommunicationService.java        | 32 +++++++++---
 ...StandardHashiCorpVaultCommunicationService.java | 47 +++++++++++++----
 ...andardHashiCorpVaultCommunicationServiceIT.java | 26 +++++++++
 .../HashiCorpVaultParameterValueProvider.java      | 35 ++++---------
 .../TestHashiCorpVaultParameterValueProvider.java  | 61 +++++++++++++++++++---
 nifi-stateless/nifi-stateless-assembly/README.md   |  5 +-
 6 files changed, 156 insertions(+), 50 deletions(-)

diff --git a/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/HashiCorpVaultCommunicationService.java b/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/HashiCorpVaultCommunicationService.java
index bf43268..8e9f8c5 100644
--- a/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/HashiCorpVaultCommunicationService.java
+++ b/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/HashiCorpVaultCommunicationService.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.vault.hashicorp;
 
+import java.util.Map;
 import java.util.Optional;
 
 /**
@@ -45,22 +46,41 @@ public interface HashiCorpVaultCommunicationService {
     byte[] decrypt(String transitPath, String cipherText);
 
     /**
-     * Writes a secret using Vault's unversioned Key/Value Secrets Engine.
+     * Writes a single secret value using Vault's unversioned Key/Value Secrets Engine.
      *
      * @see <a href="https://www.vaultproject.io/api-docs/secret/kv/kv-v1">https://www.vaultproject.io/api-docs/secret/kv/kv-v1</a>
      * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
-     * @param key The secret key
+     * @param secretKey The secret key
      * @param value The secret value
      */
-    void writeKeyValueSecret(String keyValuePath, String key, String value);
+    void writeKeyValueSecret(String keyValuePath, String secretKey, String value);
 
     /**
-     * Reads a secret from Vault's unversioned Key/Value Secrets Engine.
+     * Reads a single secret value from Vault's unversioned Key/Value Secrets Engine.
      *
      * @see <a href="https://www.vaultproject.io/api-docs/secret/kv/kv-v1">https://www.vaultproject.io/api-docs/secret/kv/kv-v1</a>
      * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
-     * @param key The secret key
+     * @param secretKey The secret key
      * @return The secret value, or empty if not found
      */
-    Optional<String> readKeyValueSecret(String keyValuePath, String key);
+    Optional<String> readKeyValueSecret(String keyValuePath, String secretKey);
+
+    /**
+     * Writes a secret with multiple key/value pairs using Vault's unversioned Key/Value Secrets Engine.
+     *
+     * @see <a href="https://www.vaultproject.io/api-docs/secret/kv/kv-v1">https://www.vaultproject.io/api-docs/secret/kv/kv-v1</a>
+     * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
+     * @param keyValues A map from key to value for keys/values that should be stored in the secret
+     */
+    void writeKeyValueSecretMap(String keyValuePath, String secretKey, Map<String, String> keyValues);
+
+    /**
+     * Reads a secret with multiple key/value pairs from Vault's unversioned Key/Value Secrets Engine.
+     *
+     * @see <a href="https://www.vaultproject.io/api-docs/secret/kv/kv-v1">https://www.vaultproject.io/api-docs/secret/kv/kv-v1</a>
+     * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
+     * @param secretKey The secret key
+     * @return A map from key to value from the secret key/values, or an empty map if not found
+     */
+    Map<String, String> readKeyValueSecretMap(String keyValuePath, String secretKey);
 }
diff --git a/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationService.java b/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationService.java
index 21c9213..3450843 100644
--- a/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationService.java
+++ b/nifi-commons/nifi-vault-utils/src/main/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationService.java
@@ -31,8 +31,10 @@ import org.springframework.vault.support.Ciphertext;
 import org.springframework.vault.support.Plaintext;
 import org.springframework.vault.support.VaultResponseSupport;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 
 import static org.springframework.vault.core.VaultKeyValueOperationsSupport.KeyValueBackend.KV_1;
@@ -41,7 +43,6 @@ import static org.springframework.vault.core.VaultKeyValueOperationsSupport.KeyV
  * Implements the VaultCommunicationService using Spring Vault
  */
 public class StandardHashiCorpVaultCommunicationService implements HashiCorpVaultCommunicationService {
-    private final HashiCorpVaultConfiguration vaultConfiguration;
     private final VaultTemplate vaultTemplate;
     private final VaultTransitOperations transitOperations;
     private final Map<String, VaultKeyValueOperations> keyValueOperationsMap;
@@ -52,7 +53,7 @@ public class StandardHashiCorpVaultCommunicationService implements HashiCorpVaul
      * @throws HashiCorpVaultConfigurationException If the configuration was invalid
      */
     public StandardHashiCorpVaultCommunicationService(final PropertySource<?>... propertySources) throws HashiCorpVaultConfigurationException {
-        vaultConfiguration = new HashiCorpVaultConfiguration(propertySources);
+        final HashiCorpVaultConfiguration vaultConfiguration = new HashiCorpVaultConfiguration(propertySources);
 
         vaultTemplate = new VaultTemplate(vaultConfiguration.vaultEndpoint(),
                 ClientHttpRequestFactoryFactory.create(vaultConfiguration.clientOptions(), vaultConfiguration.sslConfiguration()),
@@ -82,32 +83,58 @@ public class StandardHashiCorpVaultCommunicationService implements HashiCorpVaul
     }
 
     /**
-     * Writes the value to the "value" key of the secret with the path [keyValuePath]/[key].
+     * Writes the value to the "value" secretKey of the secret with the path [keyValuePath]/[secretKey].
      * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
-     * @param key The secret key
+     * @param secretKey The secret secretKey
      * @param value The secret value
      */
     @Override
-    public void writeKeyValueSecret(final String keyValuePath, final String key, final String value) {
+    public void writeKeyValueSecret(final String keyValuePath, final String secretKey, final String value) {
+        Objects.requireNonNull(keyValuePath, "Vault K/V path must be specified");
+        Objects.requireNonNull(secretKey, "Secret secretKey must be specified");
+        Objects.requireNonNull(value, "Secret value must be specified");
         final VaultKeyValueOperations keyValueOperations = keyValueOperationsMap
                 .computeIfAbsent(keyValuePath, path -> vaultTemplate.opsForKeyValue(path, KV_1));
-        keyValueOperations.put(key, new SecretData(value));
+        keyValueOperations.put(secretKey, new SecretData(value));
     }
 
     /**
-     * Returns the value of the "value" key from the secret at the path [keyValuePath]/[key].
+     * Returns the value of the "value" secretKey from the secret at the path [keyValuePath]/[secretKey].
      * @param keyValuePath The Vault path to use for the configured Key/Value v1 Secrets Engine
-     * @param key The secret key
+     * @param secretKey The secret secretKey
      * @return The value of the secret
      */
     @Override
-    public Optional<String> readKeyValueSecret(final String keyValuePath, final String key) {
+    public Optional<String> readKeyValueSecret(final String keyValuePath, final String secretKey) {
+        Objects.requireNonNull(keyValuePath, "Vault K/V path must be specified");
+        Objects.requireNonNull(secretKey, "Secret secretKey must be specified");
         final VaultKeyValueOperations keyValueOperations = keyValueOperationsMap
                 .computeIfAbsent(keyValuePath, path -> vaultTemplate.opsForKeyValue(path, KV_1));
-        final VaultResponseSupport<SecretData> response = keyValueOperations.get(key, SecretData.class);
+        final VaultResponseSupport<SecretData> response = keyValueOperations.get(secretKey, SecretData.class);
         return response == null ? Optional.empty() : Optional.ofNullable(response.getRequiredData().getValue());
     }
 
+    @Override
+    public void writeKeyValueSecretMap(final String keyValuePath, final String secretKey, final Map<String, String> keyValues) {
+        Objects.requireNonNull(keyValuePath, "Vault K/V path must be specified");
+        Objects.requireNonNull(secretKey, "Secret secretKey must be specified");
+        Objects.requireNonNull(keyValues, "Key/values map must be specified");
+        if (keyValues.isEmpty()) {
+            return;
+        }
+        final VaultKeyValueOperations keyValueOperations = keyValueOperationsMap
+                .computeIfAbsent(keyValuePath, path -> vaultTemplate.opsForKeyValue(path, KV_1));
+        keyValueOperations.put(secretKey, keyValues);
+    }
+
+    @Override
+    public Map<String, String> readKeyValueSecretMap(final String keyValuePath, final String key) {
+        final VaultKeyValueOperations keyValueOperations = keyValueOperationsMap
+                .computeIfAbsent(keyValuePath, path -> vaultTemplate.opsForKeyValue(path, KV_1));
+        final VaultResponseSupport<Map> response = keyValueOperations.get(key, Map.class);
+        return response == null ? Collections.emptyMap() : (Map<String, String>) response.getRequiredData();
+    }
+
     private static class SecretData {
         private final String value;
 
diff --git a/nifi-commons/nifi-vault-utils/src/test/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationServiceIT.java b/nifi-commons/nifi-vault-utils/src/test/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationServiceIT.java
index f347801..adca355 100644
--- a/nifi-commons/nifi-vault-utils/src/test/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationServiceIT.java
+++ b/nifi-commons/nifi-vault-utils/src/test/java/org/apache/nifi/vault/hashicorp/StandardHashiCorpVaultCommunicationServiceIT.java
@@ -21,6 +21,8 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 
@@ -67,6 +69,9 @@ public class StandardHashiCorpVaultCommunicationServiceIT {
         assertEquals(plaintext, new String(decrypted, StandardCharsets.UTF_8));
     }
 
+    /**
+     * Run <code>vault kv get kv/key</code> to see the secret
+     */
     @Test
     public void testReadWriteSecret() {
         final String key = "key";
@@ -77,4 +82,25 @@ public class StandardHashiCorpVaultCommunicationServiceIT {
         final String resultValue = vcs.readKeyValueSecret("kv", key).orElseThrow(() -> new NullPointerException("Missing secret for kv/key"));
         assertEquals(value, resultValue);
     }
+
+    /**
+     * Run <code>vault kv get kv/secret</code> to see the secret
+     */
+    @Test
+    public void testReadWriteSecretMap() {
+        final String secretKey = "secret";
+        final String key = "key";
+        final String value = "value";
+        final String key2 = "key2";
+        final String value2 = "value2";
+
+        final Map<String, String> keyValues = new HashMap<>();
+        keyValues.put(key, value);
+        keyValues.put(key2, value2);
+
+        vcs.writeKeyValueSecretMap("kv", secretKey, keyValues);
+
+        final Map<String, String> resultMap = vcs.readKeyValueSecretMap("kv", secretKey);
+        assertEquals(keyValues, resultMap);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/main/java/org/apache/nifi/stateless/parameter/HashiCorpVaultParameterValueProvider.java b/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/main/java/org/apache/nifi/stateless/parameter/HashiCorpVaultParameterValueProvider.java
index 2fba80c..dd97a78 100644
--- a/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/main/java/org/apache/nifi/stateless/parameter/HashiCorpVaultParameterValueProvider.java
+++ b/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/main/java/org/apache/nifi/stateless/parameter/HashiCorpVaultParameterValueProvider.java
@@ -24,16 +24,16 @@ import org.apache.nifi.vault.hashicorp.config.HashiCorpVaultConfiguration;
 import org.springframework.core.env.PropertySource;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 
 /**
  * Reads secrets from HashiCorp Vault to provide parameters.  An example of setting one such secret parameter value
  * using the Vault CLI would be:
  *
- * <code>vault kv put "${vault.kv.path}/[ParamContextName]/[ParamName]" value=[ParamValue]</code>
+ * <code>vault kv put "${vault.kv.path}/[ParamContextName]" [Param1]=[ParamValue1] [Param2]=[ParamValue2]</code>
  *
  * Here, vault.kv.path is supplied by the file specified by the "Vault Configuration File" property.
  *
@@ -45,9 +45,8 @@ import java.util.Objects;
  *      nifi.stateless.parameter.provider.Vault.properties.vault-configuration-file=./conf/bootstrap-hashicorp-vault.conf
  * </code>
  */
-public class HashiCorpVaultParameterValueProvider extends AbstractParameterValueProvider implements ParameterValueProvider {
+public class HashiCorpVaultParameterValueProvider extends AbstractSecretBasedParameterValueProvider implements ParameterValueProvider {
     private static final String KEY_VALUE_PATH = "vault.kv.path";
-    private static final String QUALIFIED_SECRET_FORMAT = "%s/%s";
     public static final PropertyDescriptor VAULT_CONFIG_FILE = new PropertyDescriptor.Builder()
             .displayName("Vault Configuration File")
             .name("vault-configuration-file")
@@ -57,24 +56,16 @@ public class HashiCorpVaultParameterValueProvider extends AbstractParameterValue
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
             .build();
 
-    private List<PropertyDescriptor> descriptors;
-
     private HashiCorpVaultCommunicationService vaultCommunicationService;
     private String path;
 
     @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return descriptors;
+    protected List<PropertyDescriptor> getAdditionalSupportedPropertyDescriptors() {
+        return Collections.singletonList(VAULT_CONFIG_FILE);
     }
 
     @Override
-    protected void init(final ParameterValueProviderInitializationContext context) {
-        super.init(context);
-
-        final List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.add(VAULT_CONFIG_FILE);
-        this.descriptors = Collections.unmodifiableList(descriptors);
-
+    protected void additionalInit(final ParameterValueProviderInitializationContext context) {
         final String vaultBootstrapConfFilename = context.getProperty(VAULT_CONFIG_FILE).getValue();
         this.configure(vaultBootstrapConfFilename);
     }
@@ -89,18 +80,10 @@ public class HashiCorpVaultParameterValueProvider extends AbstractParameterValue
         }
     }
 
-    private String getQualifiedSecretFormat(final String contextName, final String parameterName) {
-        return String.format(QUALIFIED_SECRET_FORMAT, contextName, parameterName);
-    }
-
-    @Override
-    public String getParameterValue(final String contextName, final String parameterName) {
-        return vaultCommunicationService.readKeyValueSecret(path, getQualifiedSecretFormat(contextName, parameterName)).orElse(null);
-    }
-
     @Override
-    public boolean isParameterDefined(final String contextName, final String parameterName) {
-        return getParameterValue(contextName, parameterName) != null;
+    protected String getSecretValue(final String secretName, final String keyName) {
+        final Map<String, String> keyValues = vaultCommunicationService.readKeyValueSecretMap(path, secretName);
+        return keyValues.get(keyName);
     }
 
     void setVaultCommunicationService(final HashiCorpVaultCommunicationService vaultCommunicationService) {
diff --git a/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/test/java/org/apache/nifi/stateless/parameter/TestHashiCorpVaultParameterValueProvider.java b/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/test/java/org/apache/nifi/stateless/parameter/TestHashiCorpVaultParameterValueProvider.java
index f86e285..389a92d 100644
--- a/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/test/java/org/apache/nifi/stateless/parameter/TestHashiCorpVaultParameterValueProvider.java
+++ b/nifi-nar-bundles/nifi-hashicorp-vault-bundle/nifi-hashicorp-vault-parameter-value-provider/src/test/java/org/apache/nifi/stateless/parameter/TestHashiCorpVaultParameterValueProvider.java
@@ -27,10 +27,14 @@ import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
-import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.when;
@@ -40,6 +44,7 @@ public class TestHashiCorpVaultParameterValueProvider {
 
     private static final String PATH = "kv";
     private static final String CONFIG_FILE = "bootstrap-hashicorp-vault.conf";
+    private static final String DEFAULT_SECRET_NAME = "Test";
 
     @Mock
     private HashiCorpVaultCommunicationService vaultCommunicationService;
@@ -63,13 +68,19 @@ public class TestHashiCorpVaultParameterValueProvider {
             }
 
             @Override
-            public PropertyValue getProperty(PropertyDescriptor descriptor) {
-                return new StandardPropertyValue(CONFIG_FILE, null, null);
+            public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+                if (descriptor.equals(HashiCorpVaultParameterValueProvider.VAULT_CONFIG_FILE)) {
+                    return new StandardPropertyValue(CONFIG_FILE, null, null);
+                } else if (descriptor.equals(HashiCorpVaultParameterValueProvider.DEFAULT_SECRET_NAME)) {
+                    return new StandardPropertyValue(DEFAULT_SECRET_NAME, null, null);
+                }
+                return null;
             }
 
             @Override
             public Map<String, String> getAllProperties() {
-                return null;
+                return Stream.of(HashiCorpVaultParameterValueProvider.VAULT_CONFIG_FILE, HashiCorpVaultParameterValueProvider.DEFAULT_SECRET_NAME)
+                        .collect(Collectors.toMap(PropertyDescriptor::getName, descriptor -> getProperty(descriptor).getValue()));
             }
         });
     }
@@ -77,15 +88,53 @@ public class TestHashiCorpVaultParameterValueProvider {
     @Test
     public void testGetParameterValue() {
         final String value = "value";
-        when(vaultCommunicationService.readKeyValueSecret(PATH, "context/param")).thenReturn(Optional.of(value));
+        final Map<String, String> keyValues = new HashMap<>();
+        keyValues.put("param", value);
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, "context")).thenReturn(keyValues);
 
         assertEquals(value, parameterProvider.getParameterValue("context", "param"));
     }
 
     @Test
+    public void testGetParameterValueNoSecret() {
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, "context")).thenReturn(Collections.emptyMap());
+
+        assertNull(parameterProvider.getParameterValue("context", "param"));
+    }
+
+    @Test
+    public void testGetParameterValueNoMatchingKey() {
+        final String value = "value";
+        final Map<String, String> keyValues = new HashMap<>();
+        keyValues.put("differentParam", value);
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, "context")).thenReturn(keyValues);
+
+        assertNull(parameterProvider.getParameterValue("context", "param"));
+    }
+
+    @Test
+    public void testGetParameterValueDefaultSecret() {
+        final String value = "value";
+        final String defaultValue = "defaultValue";
+        final Map<String, String> keyValues = new HashMap<>();
+        keyValues.put("differentParam", value);
+
+        final Map<String, String> defaultKeyValues = new HashMap<>();
+        defaultKeyValues.put("param", defaultValue);
+
+        // Although 'context' does not contain 'param', the default secret does, so it should fall back to that
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, "context")).thenReturn(keyValues);
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, DEFAULT_SECRET_NAME)).thenReturn(defaultKeyValues);
+
+        assertEquals(defaultValue, parameterProvider.getParameterValue("context", "param"));
+    }
+
+    @Test
     public void testIsParameterDefined() {
         final String value = "value";
-        when(vaultCommunicationService.readKeyValueSecret(PATH, "context/param")).thenReturn(Optional.of(value));
+        final Map<String, String> keyValues = new HashMap<>();
+        keyValues.put("param", value);
+        when(vaultCommunicationService.readKeyValueSecretMap(PATH, "context")).thenReturn(keyValues);
 
         assertTrue(parameterProvider.isParameterDefined("context", "param"));
     }
diff --git a/nifi-stateless/nifi-stateless-assembly/README.md b/nifi-stateless/nifi-stateless-assembly/README.md
index 9910dbe..2cddb33 100644
--- a/nifi-stateless/nifi-stateless-assembly/README.md
+++ b/nifi-stateless/nifi-stateless-assembly/README.md
@@ -503,17 +503,18 @@ via the `./conf/bootstrap-hashicorp-vault.conf` file, which comes with NiFi.
 An example of creating a single secret in the correct format is:
 
 ```
-vault kv put "nifi-kv/Context/param" value=my-vault-value
+vault kv put "nifi-kv/Context" param=value param2=value2
 ```
 
 In this example, `nifi-kv` would be supplied by the `vault.kv.path` property in the `bootstrap-hashicorp-vault.conf` file, 
-`Context` is the name of a Parameter Context, and `param` is the name of the parameter whose value should be retrieved from the Vault server.
+`Context` is the name of a mapped Parameter Context, and `param` and `param2` are the names of the parameters whose values should be retrieved from the Vault server.
 
 This Parameter Provider requires the following properties:
 
 | Property Name | Description | Example Value |
 |---------------|-------------|---------------|
 | nifi.stateless.parameter.provider.\<key>.properties.vault-configuration-file | The filename of a configuration file specifying the Vault settings | ./conf/bootstrap-hashicorp-vault.conf |
+| nifi.stateless.parameter.provider.\<key>.properties.default-secret-name | The default K/V secret name to use.  This secret represents a default Parameter Context if there is not a matching key within the mapped Parameter Context secret. | `Default` |
 
 An example of configuring this provider in the dataflow configuration file is: