You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/11/01 10:03:42 UTC

[GitHub] [nifi] gresockj commented on a diff in pull request #6545: NIFI-9697: Implemented AzureKeyVaultSecretsParameterProvider and Azur…

gresockj commented on code in PR #6545:
URL: https://github.com/apache/nifi/pull/6545#discussion_r1010228894


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";

Review Comment:
   This seems like it could be private.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")

Review Comment:
   Perhaps "Vault URI of the Key Vault that contains the secrets" would match the Azure lingo a little closer



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")

Review Comment:
   I think this should be "Tag" instead of "label".



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";

Review Comment:
   A little more self-descriptive would be `GROUP_NAME_TAG`.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(AZURE_CREDENTIALS_SERVICE);
+        props.add(KEY_VAULT_URL);
+        props.add(GROUP_NAME_PATTERN);
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public List<ParameterGroup> fetchParameters(final ConfigurationContext context) throws IOException {
+        final SecretClient secretClient = configureSecretClient(context);
+        final List<KeyVaultSecret> secrets = getAllSecrets(secretClient);
+        final List<ParameterGroup> groups = getParameterGroupsFromSecrets(context, secrets);
+        return groups;
+    }
+
+    @Override
+    public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger) {
+        final List<ConfigVerificationResult> results = new ArrayList<>();
+
+        try {
+            final List<ParameterGroup> parameterGroups = fetchParameters(context);
+            int parameterCount = 0;
+            for (final ParameterGroup group : parameterGroups) {
+                parameterCount += group.getParameters().size();
+            }
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.SUCCESSFUL)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation(String.format("Fetched secret keys [%d] as parameters, across groups [%d]",
+                            parameterCount, parameterGroups.size()))
+                    .build());
+        } catch (final Exception e) {
+            verificationLogger.error("Failed to fetch parameters", e);
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.FAILED)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation("Failed to fetch parameters: " + e.getMessage())
+                    .build());
+        }
+        return results;
+    }
+
+    private List<KeyVaultSecret> getAllSecrets(final SecretClient secretClient) {
+        final List<KeyVaultSecret> secrets = new ArrayList<>();
+
+        for (final SecretProperties secretProperties : secretClient.listPropertiesOfSecrets()) {
+            KeyVaultSecret secretWithValue = secretClient.getSecret(secretProperties.getName(), secretProperties.getVersion());
+            secrets.add(secretWithValue);
+        }
+
+        return secrets;
+    }
+
+    private List<ParameterGroup> getParameterGroupsFromSecrets(final ConfigurationContext context, final List<KeyVaultSecret> secrets) {
+        final Map<String, List<Parameter>> nameToParametersMap = new HashMap<>();
+        for (final KeyVaultSecret secret: secrets) {
+            final String parameterName = secret.getName();
+            final String parameterValue = secret.getValue();
+
+            final String parameterGroupName = secret.getProperties().getTags().get(PARAMETER_GROUP_NAME);
+            if (parameterGroupName == null) {
+                getLogger().warn("Secret with parameter name [{}] not recognized as a valid parameter as it is " +

Review Comment:
   Recommend phrasing `Secret with parameter name [{}] is not recognized as a valid parameter since it does not have the [{}] Tag`



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/StandardAzureCredentialsControllerService.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.services.azure;
+
+import com.azure.core.credential.TokenCredential;
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides credentials used by Azure clients.
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "security", "credentials", "provider", "session"})
+@CapabilityDescription("Provide credentials to use with an Azure client.")
+public class StandardAzureCredentialsControllerService extends AbstractControllerService implements AzureCredentialsService {
+    public static AllowableValue DEFAULT_CREDENTIAL = new AllowableValue("default-credential",
+            "Default Credential",
+            "Uses default credential chain. It first checks environment variables, before trying managed identity.");
+    public static AllowableValue MANAGED_IDENTITY = new AllowableValue("managed-identity",
+            "Managed Identity",
+            "Azure Virtual Machine Managed Identity (it can only be used when NiFi is running on Azure)");
+    public static final PropertyDescriptor CREDENTIAL_CONFIGURATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("credential-configuration-strategy")
+            .displayName("Credential Configuration Strategy")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .sensitive(false)
+            .allowableValues(DEFAULT_CREDENTIAL, MANAGED_IDENTITY)
+            .defaultValue(DEFAULT_CREDENTIAL.toString())
+            .build();
+
+    public static final PropertyDescriptor MANAGED_IDENTITY_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("managed-identity-client-id")
+            .displayName("Managed Identity Client ID")
+            .description("Client ID of the managed identity. The property is required when User Assigned Managed Identity is used for authentication. " +
+                    "It must be empty in case of System Assigned Managed Identity.")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(CREDENTIAL_CONFIGURATION_STRATEGY, MANAGED_IDENTITY)
+            .build();
+
+    private static final List<PropertyDescriptor> properties;

Review Comment:
   `PROPERTIES`



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService:
##########
@@ -19,3 +19,4 @@ org.apache.nifi.services.azure.storage.ADLSCredentialsControllerService
 org.apache.nifi.services.azure.cosmos.document.AzureCosmosDBClientService
 org.apache.nifi.services.azure.storage.AzureStorageEmulatorCredentialsControllerService
 org.apache.nifi.services.azure.storage.AzureStorageCredentialsControllerService_v12
+org.apache.nifi.services.azure.StandardAzureCredentialsControllerService

Review Comment:
   A bit unfortunate that the existing standard here for Azure extensions seems to be to group everything in the `nifi-azure-processors` module.  I think ideally we'd want a separate `nifi-azure-service-api` and `nifi-azure-service-api-nar`.  However, since the precedent is to store Azure services in this module, I think you're right to have it here for now.  Perhaps this lends itself to a restructuring for NiFi 2.0!



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(AZURE_CREDENTIALS_SERVICE);
+        props.add(KEY_VAULT_URL);
+        props.add(GROUP_NAME_PATTERN);
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public List<ParameterGroup> fetchParameters(final ConfigurationContext context) throws IOException {
+        final SecretClient secretClient = configureSecretClient(context);
+        final List<KeyVaultSecret> secrets = getAllSecrets(secretClient);
+        final List<ParameterGroup> groups = getParameterGroupsFromSecrets(context, secrets);
+        return groups;
+    }
+
+    @Override
+    public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger) {
+        final List<ConfigVerificationResult> results = new ArrayList<>();
+
+        try {
+            final List<ParameterGroup> parameterGroups = fetchParameters(context);
+            int parameterCount = 0;
+            for (final ParameterGroup group : parameterGroups) {
+                parameterCount += group.getParameters().size();
+            }
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.SUCCESSFUL)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation(String.format("Fetched secret keys [%d] as parameters, across groups [%d]",
+                            parameterCount, parameterGroups.size()))
+                    .build());
+        } catch (final Exception e) {
+            verificationLogger.error("Failed to fetch parameters", e);
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.FAILED)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation("Failed to fetch parameters: " + e.getMessage())
+                    .build());
+        }
+        return results;
+    }
+
+    private List<KeyVaultSecret> getAllSecrets(final SecretClient secretClient) {
+        final List<KeyVaultSecret> secrets = new ArrayList<>();
+
+        for (final SecretProperties secretProperties : secretClient.listPropertiesOfSecrets()) {
+            KeyVaultSecret secretWithValue = secretClient.getSecret(secretProperties.getName(), secretProperties.getVersion());
+            secrets.add(secretWithValue);
+        }
+
+        return secrets;
+    }
+
+    private List<ParameterGroup> getParameterGroupsFromSecrets(final ConfigurationContext context, final List<KeyVaultSecret> secrets) {
+        final Map<String, List<Parameter>> nameToParametersMap = new HashMap<>();
+        for (final KeyVaultSecret secret: secrets) {
+            final String parameterName = secret.getName();
+            final String parameterValue = secret.getValue();
+
+            final String parameterGroupName = secret.getProperties().getTags().get(PARAMETER_GROUP_NAME);
+            if (parameterGroupName == null) {
+                getLogger().warn("Secret with parameter name [{}] not recognized as a valid parameter as it is " +
+                                "missing the [{}] tag", parameterName, PARAMETER_GROUP_NAME);
+                continue;
+            }
+
+            final Pattern groupNamePattern = Pattern.compile(context.getProperty(GROUP_NAME_PATTERN).getValue());
+
+            if (!groupNamePattern.matcher(parameterGroupName).matches()) {
+                getLogger().debug("Secret [{}] with tag [{}] does not match the group name pattern {}", parameterName,
+                        parameterGroupName, groupNamePattern);
+                continue;
+            }
+
+            final List<Parameter> parameters = nameToParametersMap.getOrDefault(parameterGroupName, new ArrayList<>());
+
+            parameters.add(createParameter(parameterName, parameterValue));
+
+            nameToParametersMap.put(parameterGroupName, parameters);

Review Comment:
   Slightly more efficient would be:
   ```suggestion
               nameToParametersMap.computeIfAbsent(parameterGroupName, groupName -> new ArrayList<>()).add(createParameter(parameterName, parameterValue));
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(AZURE_CREDENTIALS_SERVICE);
+        props.add(KEY_VAULT_URL);
+        props.add(GROUP_NAME_PATTERN);
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public List<ParameterGroup> fetchParameters(final ConfigurationContext context) throws IOException {
+        final SecretClient secretClient = configureSecretClient(context);
+        final List<KeyVaultSecret> secrets = getAllSecrets(secretClient);
+        final List<ParameterGroup> groups = getParameterGroupsFromSecrets(context, secrets);
+        return groups;
+    }
+
+    @Override
+    public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger) {
+        final List<ConfigVerificationResult> results = new ArrayList<>();
+
+        try {
+            final List<ParameterGroup> parameterGroups = fetchParameters(context);
+            int parameterCount = 0;
+            for (final ParameterGroup group : parameterGroups) {
+                parameterCount += group.getParameters().size();
+            }
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.SUCCESSFUL)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation(String.format("Fetched secret keys [%d] as parameters, across groups [%d]",
+                            parameterCount, parameterGroups.size()))
+                    .build());
+        } catch (final Exception e) {
+            verificationLogger.error("Failed to fetch parameters", e);
+            results.add(new ConfigVerificationResult.Builder()
+                    .outcome(ConfigVerificationResult.Outcome.FAILED)
+                    .verificationStepName("Fetch Parameters")
+                    .explanation("Failed to fetch parameters: " + e.getMessage())
+                    .build());
+        }
+        return results;
+    }
+
+    private List<KeyVaultSecret> getAllSecrets(final SecretClient secretClient) {
+        final List<KeyVaultSecret> secrets = new ArrayList<>();
+
+        for (final SecretProperties secretProperties : secretClient.listPropertiesOfSecrets()) {
+            KeyVaultSecret secretWithValue = secretClient.getSecret(secretProperties.getName(), secretProperties.getVersion());
+            secrets.add(secretWithValue);
+        }
+
+        return secrets;
+    }
+
+    private List<ParameterGroup> getParameterGroupsFromSecrets(final ConfigurationContext context, final List<KeyVaultSecret> secrets) {
+        final Map<String, List<Parameter>> nameToParametersMap = new HashMap<>();
+        for (final KeyVaultSecret secret: secrets) {
+            final String parameterName = secret.getName();
+            final String parameterValue = secret.getValue();
+
+            final String parameterGroupName = secret.getProperties().getTags().get(PARAMETER_GROUP_NAME);
+            if (parameterGroupName == null) {
+                getLogger().warn("Secret with parameter name [{}] not recognized as a valid parameter as it is " +

Review Comment:
   `WARN` is too strong here, since it is expected that there will probably be secrets without the `group-name` Tag that the user doesn't intend to be a NiFi parameter.  I recommend changing this to `DEBUG`.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()
+            .name("key-vault-url")
+            .displayName("Key Vault URL")
+            .description("Key Vault URL to fetch secrets from")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor GROUP_NAME_PATTERN = new PropertyDescriptor.Builder()
+            .name("group-name-pattern")
+            .displayName("Group Name Pattern")
+            .description("A Regular Expression matching on the 'group-name' label value that identifies Secrets whose parameters should be fetched. " +
+                    "Any secrets without a 'group-name' label value that matches this Regex will not be fetched.")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .required(true)
+            .defaultValue(".*")
+            .build();
+
+    public static final String PARAMETER_GROUP_NAME = "group-name";
+
+    private static final List<PropertyDescriptor> properties;

Review Comment:
   In keeping with Java naming standards, this should be named `PROPERTIES`.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/pom.xml:
##########
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-azure-bundle</artifactId>
+        <version>1.19.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-azure-parameter-providers</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-azure-services-api</artifactId>
+            <version>1.19.0-SNAPSHOT</version>
+            <scope>compile</scope>

Review Comment:
   I believe this needs to be `provided`



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +

Review Comment:
   I think this is a copy/paste error.  Should read something like: "Each secret becomes a Parameter, which can be mapped to a Parameter Group by adding a Tag named 'group-name'."



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-parameter-providers/src/main/java/org/apache/nifi/parameter/azure/AzureKeyVaultSecretsParameterProvider.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.parameter.azure;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+import com.azure.security.keyvault.secrets.models.KeyVaultSecret;
+import com.azure.security.keyvault.secrets.models.SecretProperties;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.parameter.AbstractParameterProvider;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.parameter.ParameterGroup;
+import org.apache.nifi.parameter.VerifiableParameterProvider;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.AzureCredentialsService;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Reads secrets from Azure Key Vault Secrets to provide parameter values.  Secrets must be created similar to the following Azure cli command: <br/><br/>
+ * <code>az keyvault secret set --vault-name &lt;your-unique-keyvault-name> --name &lt;parameter-name> --value &lt;parameter-value>
+ * --tags group-name=&lt;group-name></code> <br/><br/>
+ * @see <a href="https://learn.microsoft.com/en-us/azure/key-vault/secrets/quick-create-cli">Azure Key Vault Secrets</a>
+ */
+@Tags({"azure", "keyvault", "key", "vault", "secrets"})
+@CapabilityDescription("Fetches parameters from Azure Key Vault Secrets.  Each secret becomes a Parameter, which can map to a Parameter Context, with " +
+        "key/value pairs in the secret mapping to Parameters in the group.")
+public class AzureKeyVaultSecretsParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+    public static final PropertyDescriptor AZURE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-credentials-service")
+            .displayName("Azure Credentials Service")
+            .description("Controller service used to obtain Azure credentials to be used with Key Vault client")
+            .required(true)
+            .identifiesControllerService(AzureCredentialsService.class)
+            .build();
+    public static final PropertyDescriptor KEY_VAULT_URL = new PropertyDescriptor.Builder()

Review Comment:
   I realize the `SecretClient` name for this is `vaultUrl`, but since the Azure console calls this "Vault URI", I'd rename this to `uri`/`URI` in all places.



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

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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