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 2021/03/02 01:03:11 UTC

[GitHub] [nifi] sushilkm opened a new pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

sushilkm opened a new pull request #4863:
URL: https://github.com/apache/nifi/pull/4863


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   Implements controller service to use secrets from Azure Keyvault: Implements NIFI-8279
   
   Adds following secure controller services using credentials from keyvault secret
   - AzureKeyVaultClientService
   - AzureStorageSecureCredentialsControllerService
   - ADLSSecureCredentialsControllerService
   - AzureCosmosSecureDBClientService
   
   Added tests for
   - AzureKeyVaultUtils
   - AzureKeyVaultClientService
   - ADLSSecureCredentialsControllerService
   - AzureStorageSecureCredentialsControllerService
   - AzureCosmosSecureDBClientService
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on JDK 8?
   - [x] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] sjyang18 commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sjyang18 commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r587868912



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosDBClientService.java
##########
@@ -44,67 +37,30 @@
         "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
         " and provides access to that connection to other Cosmos DB-related components."
 )
-public class AzureCosmosDBClientService extends AbstractControllerService implements AzureCosmosDBConnectionService {
+public class AzureCosmosDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {

Review comment:
       Redundant superinterface AzureCosmosDBConnectionService for the type AzureCosmosSecureDBClientService, already defined by AbstractCosmosDBClientServiceJava(16777547)

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")

Review comment:
       meaningful & user friendly description, not just display name in description.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {

Review comment:
       Redundant superinterface AzureCosmosDBConnectionService for the type AzureCosmosSecureDBClientService, already defined by AbstractCosmosDBClientServiceJava(16777547)

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSCredentialsControllerService.java
##########
@@ -221,4 +211,4 @@ public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attribut
             setBuilderValue.accept(credentialsBuilder, value);
         }
     }
-}
+}

Review comment:
       add a new line at the end of file

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+            AzureStorageUtils.ACCOUNT_NAME_SECRET,
+            AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+            AzureStorageUtils.ACCOUNT_KEY_SECRET,
+            AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+    ));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        boolean accountKeySet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue());
+        boolean sasTokenSet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue());
+
+        if (!onlyOneSet(accountKeySet, sasTokenSet)) {
+            StringJoiner options = new StringJoiner(", ")
+                    .add(AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName())
+                    .add(AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName());
+
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("one and only one of [" + options + "] should be set")
+                    .build());
+        }
+
+        return results;
+    }
+
+    private boolean onlyOneSet(Boolean... checks) {
+        long nrOfSet = Arrays.stream(checks)
+                .filter(check -> check)
+                .count();
+
+        return nrOfSet == 1;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attributes) {
+        ADLSCredentialsDetails.Builder credentialsBuilder = ADLSCredentialsDetails.Builder.newBuilder();
+
+        setValue(credentialsBuilder,
+                AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+                PropertyValue::getValue,
+                ADLSCredentialsDetails.Builder::setEndpointSuffix);
+
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+        String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if(StringUtils.isNotBlank(accountNameSecret)) {
+            credentialsBuilder.setAccountName(
+                    keyVaultClientService.getSecret(accountNameSecret));
+        }
+        if(StringUtils.isNotBlank(accountKeySecret)) {
+            credentialsBuilder.setAccountKey(
+                    keyVaultClientService.getSecret(accountKeySecret));

Review comment:
       what if secret is not set for this key in keyvault? or key is mispelled? Does this throw exception?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;

Review comment:
       feedback I got in my previous PR was to use private for this line and use getCosmosClient method to access.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })

Review comment:
       Is this controller supporting queue?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+            AzureStorageUtils.ACCOUNT_NAME_SECRET,
+            AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+            AzureStorageUtils.ACCOUNT_KEY_SECRET,
+            AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+    ));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        boolean accountKeySet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue());
+        boolean sasTokenSet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue());
+
+        if (!onlyOneSet(accountKeySet, sasTokenSet)) {
+            StringJoiner options = new StringJoiner(", ")
+                    .add(AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName())
+                    .add(AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName());
+
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("one and only one of [" + options + "] should be set")
+                    .build());
+        }
+

Review comment:
       Should we validate the non-blank secret value for non-blank keySet in your key vault service?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })

Review comment:
       can we also support queue?




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r596012321



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;
+
+    @OnStopped
+    public final void onStopped() {
+        if (this.cosmosClient != null) {
+            try {
+                cosmosClient.close();
+            } catch(CosmosException e) {
+                getLogger().error("Closing CosmosClient Failed: " + e.getMessage(), e);
+            } finally {
+                this.cosmosClient = null;
+            }
+        }
+    }
+
+    protected void createCosmosClient(final String uri, final String accessKey, final String selectedConsistency){
+        final ConsistencyLevel cLevel;
+
+        switch(selectedConsistency) {
+            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
+                cLevel =  ConsistencyLevel.STRONG;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_CONSISTENT_PREFIX:
+                cLevel = ConsistencyLevel.CONSISTENT_PREFIX;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_BOUNDED_STALENESS:
+                cLevel = ConsistencyLevel.BOUNDED_STALENESS;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_EVENTUAL:
+                cLevel = ConsistencyLevel.EVENTUAL;
+                break;
+            default:
+                cLevel = ConsistencyLevel.SESSION;
+        }
+        this.cosmosClient = new CosmosClientBuilder()
+                .endpoint(uri)
+                .key(accessKey)
+                .consistencyLevel(cLevel)
+                .buildClient();
+    }
+
+    @Override
+    public CosmosClient getCosmosClient() {
+        return this.cosmosClient;
+    }
+
+    public void setCosmosClient(CosmosClient client) {

Review comment:
       Thanks for the reply and noting the existing usage.  As a simple change, making the `cosmosClient` member variable `protected` would allow assigning it in `MockTestBase`, then this method could be removed.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594776492



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {

Review comment:
       cleaned

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        this.uriSecret = context.getProperty(URI_SECRET).getValue();
+        this.accessKeySecret = context.getProperty(DB_ACCESS_KEY_SECRET).getValue();
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();
+        this.keyVaultClientService = context.getProperty(
+                KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        createCosmosClient(
+                getURI(),
+                getAccessKey(),
+                getConsistencyLevel()
+        );
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(KEYVAULT_CONNECTION_SERVICE);
+        descriptors.add(URI_SECRET);
+        descriptors.add(DB_ACCESS_KEY_SECRET);
+        descriptors.add(AzureCosmosDBUtils.CONSISTENCY);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public String getURI() {
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+        if (StringUtils.isBlank(uriSecret)) {

Review comment:
       cleaned




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594538586



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("Storage Account Key Secret Name")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")
+            .displayName("SAS Token Secret Name")
+            .description("SAS Token Secret Name")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+
+    public static final PropertyDescriptor ADLS_ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AzureStorageUtils.ENDPOINT_SUFFIX)
+            .displayName("Endpoint Suffix")
+            .description(
+                "Storage accounts in public Azure always use a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain circumstances (like Azure Stack or non-public Azure regions).")
+            .required(true)
+            .defaultValue("dfs.core.windows.net")
+            .build();

Review comment:
       it is ok to be blank, default is blank.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594777257



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;
+
+    @OnStopped
+    public final void onStopped() {
+        if (this.cosmosClient != null) {
+            try {
+                cosmosClient.close();
+            } catch(CosmosException e) {
+                getLogger().error("Closing CosmosClient Failed: " + e.getMessage(), e);
+            } finally {
+                this.cosmosClient = null;
+            }
+        }
+    }
+
+    protected void createCosmosClient(final String uri, final String accessKey, final String selectedConsistency){
+        final ConsistencyLevel cLevel;
+
+        switch(selectedConsistency) {
+            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
+                cLevel =  ConsistencyLevel.STRONG;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_CONSISTENT_PREFIX:
+                cLevel = ConsistencyLevel.CONSISTENT_PREFIX;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_BOUNDED_STALENESS:
+                cLevel = ConsistencyLevel.BOUNDED_STALENESS;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_EVENTUAL:
+                cLevel = ConsistencyLevel.EVENTUAL;
+                break;
+            default:
+                cLevel = ConsistencyLevel.SESSION;
+        }
+        this.cosmosClient = new CosmosClientBuilder()
+                .endpoint(uri)
+                .key(accessKey)
+                .consistencyLevel(cLevel)
+                .buildClient();
+    }
+
+    @Override
+    public CosmosClient getCosmosClient() {
+        return this.cosmosClient;
+    }
+
+    public void setCosmosClient(CosmosClient client) {

Review comment:
       This code was already there, it was just moved to another file in current PR.
   I am ok to update, can you suggest an example how do you want me to update this.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] jfrazee commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
jfrazee commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-1009496425


   @exceptionfactory I think when this started the sensitive parameter provider work hadn't been sketched out yet. I think it is a good intermediate solution.


-- 
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



[GitHub] [nifi] jfrazee commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
jfrazee commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r629539907



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")

Review comment:
       ```suggestion
               .description("The name of the Azure Key Vault secret containing the Azure Storage account " +
                       "name. The Controller Service will use this name to get the value of the secret from Key Vault.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Key.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")
+            .displayName("SAS Token Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage SAS Token.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")

Review comment:
       ```suggestion
               .displayName("Key Vault Connection Service")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Key.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")

Review comment:
       ```suggestion
               .name("storage-sas-token-secret")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Key.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")
+            .displayName("SAS Token Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage SAS Token.")

Review comment:
       ```suggestion
               .description("The name of the Azure Key Vault secret containing the Azure Storage SAS Token. " +
                       "The Controller Service will use this name to get the value of the secret from Key Vault.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+
+/**
+ *
+ * Service interface to provide KeyVault Secrets.
+ */
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    /**
+     * Get Secret Value from KeyVault for requested secret name

Review comment:
       ```suggestion
        * Get Secret Value from Azure Key Vault for the requested secret name
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
##########
@@ -109,6 +109,17 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-security-keyvault-secrets</artifactId>
+            <version>4.2.3</version>

Review comment:
       Can we add a property to nifi-nar-bundles/nifi-azure-bundle/pom.xml for this?
   ```suggestion
               <version>${azure.keyvault.secrets.version}</version>
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Key.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")
+            .displayName("SAS Token Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage SAS Token.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("The controller service will get the value of secrets from Keyvault. " +
+                "Provide the name of keyvault controller service.")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+
+    public static final PropertyDescriptor ADLS_ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AzureStorageUtils.ENDPOINT_SUFFIX)
+            .displayName("Endpoint Suffix")
+            .description(
+                "Storage accounts in public Azure always use a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain circumstances (like Azure Stack or non-public Azure regions).")
+            .required(true)
+            .defaultValue("dfs.core.windows.net")

Review comment:
       Maybe add:
   ```java
       public static final String DEFAULT_ADLS_ENDPOINT_SUFFIX = "dfs.core.windows.net";
   ```
   and change this to:
   ```suggestion
               .defaultValue(DEFAULT_ADLS_ENDPOINT_SUFFIX)
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {

Review comment:
       This needs to be private or we need to have some kind of wrapper (note the latter should make testing easy if this is the motivation). Having return types from 3rd party SDKs is problematic.
   ```suggestion
       private SecretClient getKeyVaultSecretClient() {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.
+ */
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,55 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The controller service will get the value of secret from Keyvault. " +
+                    "Provide the name of secret which stores Storage Account Key.")

Review comment:
       ```suggestion
               .description("The name of the Azure Key Vault secret containing the Azure Storage account " +
                       " access key. The Controller Service will use this name to get the value of the secret from " +
                       " Key Vault.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));

Review comment:
       ```suggestion
               logger.info("Secret cache enabled with cacheSize: {} and cacheTTL: {} secs", cacheSize, cacheTTL);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);

Review comment:
       If we're confident these will never be null because of defaults then these can be primitives, otherwise we should check whether they're null below when checking `cacheSize > 0` and `cacheTTL > 0`.
   ```suggestion
           final int cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
           final long cacheTTL = context.getProperty(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE).asTimePeriod(TimeUnit.SECONDS);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }

Review comment:
       We should handle null `secretName`.
   ```suggestion
       private String getSecretFromKeyVault(String secretName) {
           return Optional.ofNullable(secretName)
               .map(SecretClient::getSecret)
               .map(KeyVaultSecret::getValue)
               .orElse(null);
        }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)
+                || StringUtils.isBlank(clientSecret)
+                || StringUtils.isBlank(tenantID))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("all of " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.TENANT_ID.getDisplayName() +" are required")
+                    .build());
+        }
+        return results;
+    }
+
+    protected void createKeyVaultSecretClient(){

Review comment:
       In general we don't need to maintain a lot of these values as fields in the class. We can simplify a lot by removing most of the private fields and just passing the context into this:
   ```suggestion
       private protected SecretClient createKeyVaultSecretClient(final ConfigurationContext context) {
   ```
   and then just keeping track of the secret client and setting it in the `onEnabled()`:
   ```java
           this.keyVaultSecretClient = createKeyVaultSecretClient(context);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")

Review comment:
       ```suggestion
               .displayName("Key Vault Name")
               .description("The name of the Azure Key Vault to get secrets from.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;

Review comment:
       We don't need to keep these around for anything after creating the `SecretClient` so in the `onEnabled()` we should just set `keyVaultSecretClient` and not store these.
   ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.
+ */
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")

Review comment:
       ```suggestion
               .description("The Azure Service Principal Client ID to use for authentication.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .defaultValue(".vault.azure.net")
+            .sensitive(false)
+            .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+            .name("azure-use-managed-identity")
+            .displayName("Use Azure Managed Identity")
+            .description("Choose whether or not to use the managed identity of Azure VM/VMSS. " +
+                    "If using managed identity user will not be required to provide service principal details")
+            .required(false)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CACHE_SIZE = new PropertyDescriptor.Builder()
+            .name("cache-size")
+            .displayName("Cache size")
+            .description("Maximum number of secrets to cache. Zero disables the cache.")
+            .required(true)
+            .defaultValue("10")
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CACHE_TTL_AFTER_WRITE = new PropertyDescriptor.Builder()
+            .name("cache-ttl-after-write")
+            .displayName("Cache TTL after write")
+            .description("The cache TTL (time-to-live) or how long to keep secret in the cache after it was written.")
+            .required(true)
+            .defaultValue("600 secs")

Review comment:
       Can we change this to 5 minutes?
   ```suggestion
               .defaultValue("5 mins")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.
+ */
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .defaultValue(".vault.azure.net")

Review comment:
       Can you add a private constant for this? (At some point we need to centralize all the various endpoints, but don't need to worry about that here.)
   ```suggestion
               .defaultValue(DEFAULT_KEYVAULT_ENDPOINT_SUFFIX)
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .defaultValue(".vault.azure.net")
+            .sensitive(false)
+            .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+            .name("azure-use-managed-identity")
+            .displayName("Use Azure Managed Identity")
+            .description("Choose whether or not to use the managed identity of Azure VM/VMSS. " +
+                    "If using managed identity user will not be required to provide service principal details")
+            .required(false)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();

Review comment:
       Can we change this to be something like "Authentication Method" with "Managed Identity" and "Service Principal" as allowable values so we can use dependent properties (see [DependOnProperties.java](https://github.com/apache/nifi/blob/rel/nifi-1.13.2/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/DependOnProperties.java) for an example) and have the choices be more maintainable as choices are added or removed.
   
   ```suggestion
       private static final AllowableValue SERVICE_PRINCIPAL = AllowableValue("service-principal", "Service Principal", "Use Service Principal authentication to authenticate to Azure Key Vault.");
   
       private static final AllowableValue MANAGED_IDENTITY = AllowableValue("managed-identity", "Managed Identity", "Use a Managed Identity to authenticate to Azure Key Vault.");
   
       public static final PropertyDescriptor AUTH_METHOD = new PropertyDescriptor.Builder()
                .name("azure-keyvault-auth-method")
                .displayName("Authentication Method")
                .description("How to authentication to Azure Key Vault")
                .required(true)
                .defaultValue(SERVICE_PRINCIPAL)
                .allowableValues(SERVICE_PRINCIPAL, MANAGED_IDENTITY)
                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
                .build();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")

Review comment:
       ```suggestion
               .displayName("Endpoint Suffix")
               .description("The Endpoint or FQDN Suffix or to use when connecting Azure Key Vault." +
                       "Override this endpoint suffix with a different suffix in certain " +
                       "circumstances (like Azure Stack or non-public Azure regions).")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();

Review comment:
       I think these are still within the allowed line lengths so we probably don't need line breaks here:
   ```suggestion
           this.keyVaultName = context.getProperty(AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
           this.servicePrincipalClientID = context.getProperty(AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
           this.servicePrincipalClientSecret = context.getProperty(AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
           this.tenantID = context.getProperty(AzureKeyVaultUtils.TENANT_ID).getValue();
           this.endPointSuffix = context.getProperty(AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
           this.useManagedIdentity = context.getProperty(AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+            AzureStorageUtils.ACCOUNT_NAME_SECRET,
+            AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+            AzureStorageUtils.ACCOUNT_KEY_SECRET,
+            AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+    ));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        boolean accountKeySet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue());
+        boolean sasTokenSet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue());
+
+        if (!onlyOneSet(accountKeySet, sasTokenSet)) {
+            StringJoiner options = new StringJoiner(", ")
+                    .add(AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName())
+                    .add(AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName());
+
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("one and only one of [" + options + "] should be set")
+                    .build());
+        }
+
+        return results;
+    }
+
+    private boolean onlyOneSet(Boolean... checks) {
+        long nrOfSet = Arrays.stream(checks)
+                .filter(check -> check)
+                .count();
+
+        return nrOfSet == 1;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attributes) {
+        ADLSCredentialsDetails.Builder credentialsBuilder = ADLSCredentialsDetails.Builder.newBuilder();
+
+        setValue(credentialsBuilder,
+                AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+                PropertyValue::getValue,
+                ADLSCredentialsDetails.Builder::setEndpointSuffix);
+
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+        String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if(StringUtils.isNotBlank(accountNameSecret)) {
+            credentialsBuilder.setAccountName(
+                    keyVaultClientService.getSecret(accountNameSecret));
+        }
+        if(StringUtils.isNotBlank(accountKeySecret)) {
+            credentialsBuilder.setAccountKey(
+                    keyVaultClientService.getSecret(accountKeySecret));
+        } else if(StringUtils.isNotBlank(sasTokenSecret)) {
+            credentialsBuilder.setSasToken(
+                    keyVaultClientService.getSecret(sasTokenSecret));
+        } else {
+            throw new IllegalArgumentException(String.format(
+                    "Either '%s' or '%s' must be defined.",
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName(),
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()));
+        }
+
+        return credentialsBuilder.build();
+    }
+
+    private <T> void setValue(

Review comment:
       This is used in multiple of the controller services, so it might be worth moving out to the utils class now or in the future.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));

Review comment:
       ```suggestion
               throw new IllegalArgumentException(String.format("Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
+        }
+
+        String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);
+
+        StorageCredentials storageCredentials;
+
+        if (StringUtils.isNotBlank(accountKeySecret)) {
+            String accountKeyValue = keyVaultClientService.getSecret(accountKeySecret);

Review comment:
       What happens if the secret value for `accountKeyValue` is empty or blank?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
+        }
+
+        String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);
+
+        StorageCredentials storageCredentials;
+
+        if (StringUtils.isNotBlank(accountKeySecret)) {
+            String accountKeyValue = keyVaultClientService.getSecret(accountKeySecret);
+            storageCredentials = new StorageCredentialsAccountAndKey(accountNameValue, accountKeyValue);
+        } else if (StringUtils.isNotBlank(sasTokenSecret)) {
+            String sasTokenValue = keyVaultClientService.getSecret(sasTokenSecret);

Review comment:
       Same question for `sasTokenValue`.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
+        }
+
+        String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);

Review comment:
       ```suggestion
           final String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        if (secretName == MOCK_SECRET_NAME) {
+            return MOCK_SECRET_VALUE;
+        } else {
+            return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+
+}
+
+public class TestAzureKeyVaultClientService {
+
+    private TestRunner testRunner;
+    private MockConnectionService kvService;
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Before
+    public void Setup() throws Exception {
+        kvService = new MockConnectionService();
+        testRunner = TestRunners.newTestRunner(NoOpProcessor.class);
+        testRunner.addControllerService("kvService", kvService);
+    }
+
+    @Test
+    public void testGetKeyVaultSecretClient() {
+        assertNotNull(kvService.getKeyVaultSecretClient());
+    }
+
+    @Test
+    public void testGetKeyVaultSecret() {
+        assertEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);
+    }
+
+    @Test
+    public void testInvalidGetKeyVaultSecret() {
+        assertNotEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_INVALID_SECRET_VALUE);
+    }
+
+    @Test
+    public void testGetSecret() {
+        assertEquals(kvService.getSecret(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);

Review comment:
       ```suggestion
           assertEquals(MOCK_SECRET_VALUE, kvService.getSecret(MOCK_SECRET_NAME));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        if (secretName == MOCK_SECRET_NAME) {
+            return MOCK_SECRET_VALUE;
+        } else {
+            return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+
+}
+
+public class TestAzureKeyVaultClientService {
+
+    private TestRunner testRunner;
+    private MockConnectionService kvService;
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Before
+    public void Setup() throws Exception {
+        kvService = new MockConnectionService();
+        testRunner = TestRunners.newTestRunner(NoOpProcessor.class);
+        testRunner.addControllerService("kvService", kvService);
+    }
+
+    @Test
+    public void testGetKeyVaultSecretClient() {
+        assertNotNull(kvService.getKeyVaultSecretClient());
+    }
+
+    @Test
+    public void testGetKeyVaultSecret() {
+        assertEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);
+    }
+
+    @Test
+    public void testInvalidGetKeyVaultSecret() {
+        assertNotEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_INVALID_SECRET_VALUE);

Review comment:
       ```suggestion
           assertNotEquals(MOCK_INVALID_SECRET_VALUE , kvService.getSecretFromKeyVault(MOCK_SECRET_NAME));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        if (secretName == MOCK_SECRET_NAME) {
+            return MOCK_SECRET_VALUE;
+        } else {
+            return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+
+}
+
+public class TestAzureKeyVaultClientService {
+
+    private TestRunner testRunner;
+    private MockConnectionService kvService;
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Before
+    public void Setup() throws Exception {

Review comment:
       ```suggestion
       public void setup() throws Exception {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }

Review comment:
        ```suggestion
       @Override
       public String getSecret(String secretName) {
           return Optional.ofNullable(secretCache)
               .map(c -> {
                   final String secretValue = c.getIfPresent(secretName);
                   if (logger.isDebugEnabled()) {
                       logger.debug("Cache miss for secret '{}'", secretName);
                   }
                   return secretValue;
               })
               .orElse(getSecretFromKeyVault(secretName));
       }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();

Review comment:
       ```suggestion
           final String keyVaultName = validationContext.getProperty(AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
           final String clientID = validationContext.getProperty(AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
           final String clientSecret = validationContext.getProperty(AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
            final String tenantID = validationContext.getProperty(AzureKeyVaultUtils.TENANT_ID).getValue();
            final boolean useManagedIdentity = validationContext.getProperty(AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")

Review comment:
       ```suggestion
                       .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() + " is required")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .defaultValue(".vault.azure.net")
+            .sensitive(false)
+            .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+            .name("azure-use-managed-identity")
+            .displayName("Use Azure Managed Identity")
+            .description("Choose whether or not to use the managed identity of Azure VM/VMSS. " +
+                    "If using managed identity user will not be required to provide service principal details")
+            .required(false)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CACHE_SIZE = new PropertyDescriptor.Builder()
+            .name("cache-size")
+            .displayName("Cache size")
+            .description("Maximum number of secrets to cache. Zero disables the cache.")
+            .required(true)
+            .defaultValue("10")
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CACHE_TTL_AFTER_WRITE = new PropertyDescriptor.Builder()
+            .name("cache-ttl-after-write")
+            .displayName("Cache TTL after write")

Review comment:
       ```suggestion
               .displayName("Cache TTL after Write")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {

Review comment:
       ```suggestion
   public class AzureKeyVaultClientService extends AbstractControllerService implements AzureKeyVaultConnectionService {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {

Review comment:
       ```suggestion
   public class ADLSSecureCredentialsControllerService extends AbstractControllerService implements ADLSCredentialsService {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")

Review comment:
       ```suggestion
               .description("The Azure Service Principal Client Secret to use when using Service Principal authentication.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .name("keyvault-uri-suffix")
+            .displayName("KeyVault URI  Suffix")
+            .description("KeyVault in public Azure always uses a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain " +
+                    "circumstances (like Azure Stack or non-public Azure regions). ")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .defaultValue(".vault.azure.net")
+            .sensitive(false)
+            .build();
+
+    public static final PropertyDescriptor USE_MANAGED_IDENTITY = new PropertyDescriptor.Builder()
+            .name("azure-use-managed-identity")
+            .displayName("Use Azure Managed Identity")
+            .description("Choose whether or not to use the managed identity of Azure VM/VMSS. " +
+                    "If using managed identity user will not be required to provide service principal details")
+            .required(false)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CACHE_SIZE = new PropertyDescriptor.Builder()
+            .name("cache-size")
+            .displayName("Cache size")

Review comment:
       ```suggestion
               .displayName("Cache Size")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."

Review comment:
       ```suggestion
   @CapabilityDescription("Provides a controller service that configures a connection to Azure Key Vault" +
           " and provides access to that connection to Azure Key Vault components."
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public final class AzureKeyVaultUtils {
+
+    public static final PropertyDescriptor KEYVAULT_NAME = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-name")
+            .displayName("KeyVault Name")
+            .description("KeyVault Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_ID = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-id")
+            .displayName("Service Principal Client ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client ID for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SP_CLIENT_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-service-principal-client-secret")
+            .displayName("Service Principal Client Secret")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Service Principal Client Secret for authentication")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor TENANT_ID = new PropertyDescriptor.Builder()
+            .name("azure-tenant-id")
+            .displayName("Azure Tenant ID")
+            .description("This processor will use Azure Service Principal for authentication. " +
+                    "Please provide Azure Tenant ID for authentication")

Review comment:
       ```suggestion
               .description("The Azure Active Directory Tenant ID to use when using Service Principal authentication.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {

Review comment:
       ```suggestion
   public class AzureStorageSecureCredentialsControllerService extends AbstractControllerService implements AzureStorageCredentialsService {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)
+                || StringUtils.isBlank(clientSecret)
+                || StringUtils.isBlank(tenantID))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("all of " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.TENANT_ID.getDisplayName() +" are required")
+                    .build());
+        }
+        return results;
+    }
+
+    protected void createKeyVaultSecretClient(){
+        String kvUri = "https://" + this.keyVaultName + this.endPointSuffix;
+
+        if (this.useManagedIdentity) {

Review comment:
       See other comment about changing this to something like `authType` and using dependent properties.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)
+                || StringUtils.isBlank(clientSecret)
+                || StringUtils.isBlank(tenantID))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("all of " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + " and " + AzureKeyVaultUtils.TENANT_ID.getDisplayName() +" are required")
+                    .build());
+        }
+        return results;
+    }
+
+    protected void createKeyVaultSecretClient(){
+        String kvUri = "https://" + this.keyVaultName + this.endPointSuffix;

Review comment:
       I think we should use `java.net.URI` or `java.net.URL` here to ensure this is a valid URI.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));

Review comment:
       ```suggestion
               throw new IllegalArgumentException(String.format("'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
+        }
+
+        String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);
+
+        StorageCredentials storageCredentials;
+
+        if (StringUtils.isNotBlank(accountKeySecret)) {
+            String accountKeyValue = keyVaultClientService.getSecret(accountKeySecret);
+            storageCredentials = new StorageCredentialsAccountAndKey(accountNameValue, accountKeyValue);
+        } else if (StringUtils.isNotBlank(sasTokenSecret)) {
+            String sasTokenValue = keyVaultClientService.getSecret(sasTokenSecret);
+            storageCredentials = new StorageCredentialsSharedAccessSignature(sasTokenValue);
+        } else {
+            throw new IllegalArgumentException(String.format(

Review comment:
       The arguments values are validated in `customValidate()`, right? If so this is more of an illegal state.
   ```suggestion
               throw new IllegalStateException(String.format(
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {

Review comment:
       Similar comment as above about using `when().thenReturn()`.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        if (secretName == MOCK_SECRET_NAME) {
+            return MOCK_SECRET_VALUE;
+        } else {
+            return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+
+}
+
+public class TestAzureKeyVaultClientService {
+
+    private TestRunner testRunner;
+    private MockConnectionService kvService;
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Before
+    public void Setup() throws Exception {
+        kvService = new MockConnectionService();
+        testRunner = TestRunners.newTestRunner(NoOpProcessor.class);
+        testRunner.addControllerService("kvService", kvService);
+    }
+
+    @Test
+    public void testGetKeyVaultSecretClient() {
+        assertNotNull(kvService.getKeyVaultSecretClient());
+    }
+
+    @Test
+    public void testGetKeyVaultSecret() {
+        assertEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);

Review comment:
       The order of the args is flipped. JUnit has the expected value on the left hand side and the actual on the right. Same for the others.
   ```suggestion
           assertEquals(MOCK_SECRET_VALUE , kvService.getSecretFromKeyVault(MOCK_SECRET_NAME));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/keyvault/TestAzureKeyVaultClientService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.keyvault;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+class MockConnectionService extends AzureKeyVaultClientService {
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        if (secretName == MOCK_SECRET_NAME) {
+            return MOCK_SECRET_VALUE;
+        } else {
+            return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+
+}
+
+public class TestAzureKeyVaultClientService {
+
+    private TestRunner testRunner;
+    private MockConnectionService kvService;
+
+    private String MOCK_SECRET_NAME = "mockSecretName1";
+    private String MOCK_SECRET_VALUE = "mockSecretValue1";
+    private String MOCK_INVALID_SECRET_VALUE = "mockInvalidSecretValue1";
+
+    @Before
+    public void Setup() throws Exception {
+        kvService = new MockConnectionService();
+        testRunner = TestRunners.newTestRunner(NoOpProcessor.class);
+        testRunner.addControllerService("kvService", kvService);
+    }
+
+    @Test
+    public void testGetKeyVaultSecretClient() {
+        assertNotNull(kvService.getKeyVaultSecretClient());
+    }
+
+    @Test
+    public void testGetKeyVaultSecret() {
+        assertEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);
+    }
+
+    @Test
+    public void testInvalidGetKeyVaultSecret() {
+        assertNotEquals(kvService.getSecretFromKeyVault(MOCK_SECRET_NAME), MOCK_INVALID_SECRET_VALUE);
+    }
+
+    @Test
+    public void testGetSecret() {
+        assertEquals(kvService.getSecret(MOCK_SECRET_NAME), MOCK_SECRET_VALUE);
+    }
+
+    @Test
+    public void testInvalidGetKeySecret() {
+        assertNotEquals(kvService.getSecret(MOCK_SECRET_NAME), MOCK_INVALID_SECRET_VALUE);

Review comment:
       ```suggestion
           assertNotEquals(MOCK_INVALID_SECRET_VALUE, kvService.getSecret(MOCK_SECRET_NAME));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/cosmos/document/TestAzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultClientService;
+
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+
+class MockKeyVaultConnectionService extends AzureKeyVaultClientService {

Review comment:
       I think we could get rid of this class and just use the `when().thenReturn()` in Mockito; e.g,
   ```java
   final SecretClient secretClient = mock(SecretClient.class);
   final AzureKeyVaultClientService keyVaultConnectionService = mock(AzureKeyVaultClientService.class);
   
   when(keyVaultConnectionService.getKeyVaultSecretClient()).thenReturn(secretClient);
   when(getSecretFromKeyVault(URI_SECRET)).thenReturn(URI_SECRET_VALUE);
   
   runner.addControllerService(keyVaultServiceName, keyVaultConnectionService);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/storage/TestADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.storage;
+
+import java.util.HashMap;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultClientService;
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+class MockKeyVaultConnectionService extends AzureKeyVaultClientService {
+
+    private static final String ACCOUNT_NAME_SECRET_NAME = "AccountNameSecret";
+    private static final String ACCOUNT_KEY_SECRET_NAME = "AccountKeySecret";
+    private static final String SAS_TOKEN_SECRET_NAME = "SasTokenSecret";
+    private static final String END_POINT_SUFFIX_SECRET_NAME = "end.point.suffix";
+
+    private String ACCOUNT_NAME_SECRET_VALUE = "AccountNameSecretValue";
+    private String ACCOUNT_KEY_SECRET_VALUE = "AccountKeySecretValue";
+    private String SAS_TOKEN_SECRET_VALUE = "SasTokenSecretValue";
+    private String END_POINT_SUFFIX_SECRET_VALUE = "end.point.suffix";
+
+    private String MOCK_INVALID_SECRET_VALUE = "INVALID_VALUE";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        switch(secretName) {
+            case ACCOUNT_NAME_SECRET_NAME:
+                return ACCOUNT_NAME_SECRET_VALUE;
+            case ACCOUNT_KEY_SECRET_NAME:
+                return ACCOUNT_KEY_SECRET_VALUE;
+            case SAS_TOKEN_SECRET_NAME:
+                return SAS_TOKEN_SECRET_VALUE;
+            case END_POINT_SUFFIX_SECRET_NAME:
+                return END_POINT_SUFFIX_SECRET_VALUE;
+            default:
+               return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+}
+
+public class TestADLSSecureCredentialsControllerService {
+
+    public static final String CREDENTIALS_SERVICE_IDENTIFIER = "secure-credentials-service";
+
+    private static final String ACCOUNT_NAME_SECRET = "AccountNameSecret";
+    private static final String ACCOUNT_KEY_SECRET = "AccountKeySecret";
+    private static final String SAS_TOKEN_SECRET = "SasTokenSecret";
+    private static final String END_POINT_SUFFIX_SECRET = "end.point.suffix";
+
+    private String ACCOUNT_NAME_SECRET_VALUE = "AccountNameSecretValue";
+    private String ACCOUNT_KEY_SECRET_VALUE = "AccountKeySecretValue";
+    private String SAS_TOKEN_SECRET_VALUE = "SasTokenSecretValue";
+    private String END_POINT_SUFFIX_SECRET_VALUE = "end.point.suffix";
+
+    private TestRunner runner;
+    private ADLSSecureCredentialsControllerService credentialsService;
+
+    @Before
+    public void setUp() throws InitializationException {
+        runner = TestRunners.newTestRunner(NoOpProcessor.class);
+        credentialsService = new ADLSSecureCredentialsControllerService();
+        runner.addControllerService(CREDENTIALS_SERVICE_IDENTIFIER, credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseAccountNameMissing() {
+        configureAccountKey();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseNoCredentialsIsSet() {
+        configureAccountName();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseBothAccountKeyAndSasTokenSpecified() {
+        configureAccountName();
+        configureAccountKey();
+        configureSasToken();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidWithEmptyEndpointSuffix() {
+        configureAccountName();
+        configureAccountKey();
+
+        runner.setProperty(credentialsService, AzureStorageUtils.ADLS_ENDPOINT_SUFFIX, "");
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidWithWhitespaceEndpointSuffix() {
+        configureAccountName();
+        configureAccountKey();
+        runner.setProperty(credentialsService, AzureStorageUtils.ADLS_ENDPOINT_SUFFIX, " ");
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testValidWithAccountNameAndAccountKey() throws InitializationException {
+        configureAccountName();
+        configureAccountKey();
+        configureKeyVaultConnectionService();
+
+        runner.assertValid(credentialsService);
+    }
+
+    @Test
+    public void testValidWithAccountNameAndSasToken() throws InitializationException {
+        configureAccountName();
+        configureSasToken();
+        configureKeyVaultConnectionService();
+
+        runner.assertValid(credentialsService);
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithAccountKey() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureAccountKey();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(ACCOUNT_NAME_SECRET_VALUE, actual.getAccountName());
+        assertEquals(ACCOUNT_KEY_SECRET_VALUE, actual.getAccountKey());
+        assertNull(actual.getSasToken());
+        assertFalse(actual.getUseManagedIdentity());
+        assertNotNull(actual.getEndpointSuffix());
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithSasToken() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureSasToken();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(ACCOUNT_NAME_SECRET_VALUE, actual.getAccountName());
+        assertEquals(SAS_TOKEN_SECRET_VALUE, actual.getSasToken());
+        assertNull(actual.getAccountKey());
+        assertFalse(actual.getUseManagedIdentity());
+        assertNotNull(actual.getEndpointSuffix());
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithSetEndpointSuffix() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureAccountKey();
+        configureEndpointSuffix();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(END_POINT_SUFFIX_SECRET, actual.getEndpointSuffix());

Review comment:
       Should this also include the gets for the account name and key?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/storage/TestADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.storage;
+
+import java.util.HashMap;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultClientService;
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+class MockKeyVaultConnectionService extends AzureKeyVaultClientService {

Review comment:
       Maybe can use `when().thenReturn()` instead here too.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/storage/TestAzureStorageSecureCredentialsControllerService.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.storage;
+
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultClientService;
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAzureStorageSecureCredentialsControllerService {

Review comment:
       This seems to be lacking the `testGetCredentials*` tests that the similar ADLS CS has. Is that needed here?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosDBClientService.java
##########
@@ -21,91 +21,45 @@
 import java.util.Collection;
 import java.util.List;
 
-import com.azure.cosmos.ConsistencyLevel;
-import com.azure.cosmos.CosmosClient;
-import com.azure.cosmos.CosmosClientBuilder;
-import com.azure.cosmos.CosmosException;
-
 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.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
-import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
 import org.apache.nifi.util.StringUtils;
 
 @Tags({"azure", "cosmos", "document", "service"})
 @CapabilityDescription(
         "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
         " and provides access to that connection to other Cosmos DB-related components."
 )
-public class AzureCosmosDBClientService extends AbstractControllerService implements AzureCosmosDBConnectionService {
+public class AzureCosmosDBClientService
+        extends AbstractCosmosDBClientService {

Review comment:
       ```suggestion
   public class AzureCosmosDBClientService extends AbstractCosmosDBClientService {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/pom.xml
##########
@@ -45,6 +45,17 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>

Review comment:
       Can we add azure-security-keyvault-secrets to the `<dependencyManagement/>` section of the root `nifi-azure-bundle/pom.xml` and just inherit the version from there (as a property)?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)

Review comment:
       ```suggestion
   @CapabilityDescription("Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +                
           " and provides access to that connection to other Cosmos DB-related components.")
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosDBClientService.java
##########
@@ -21,91 +21,45 @@
 import java.util.Collection;
 import java.util.List;
 
-import com.azure.cosmos.ConsistencyLevel;
-import com.azure.cosmos.CosmosClient;
-import com.azure.cosmos.CosmosClientBuilder;
-import com.azure.cosmos.CosmosException;
-
 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.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
-import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
 import org.apache.nifi.util.StringUtils;
 
 @Tags({"azure", "cosmos", "document", "service"})
 @CapabilityDescription(
         "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
         " and provides access to that connection to other Cosmos DB-related components."
 )
-public class AzureCosmosDBClientService extends AbstractControllerService implements AzureCosmosDBConnectionService {
+public class AzureCosmosDBClientService
+        extends AbstractCosmosDBClientService {
     private String uri;
     private String accessKey;
     private String consistencyLevel;
-    private CosmosClient cosmosClient;
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) {
         this.uri = context.getProperty(AzureCosmosDBUtils.URI).getValue();
         this.accessKey = context.getProperty(AzureCosmosDBUtils.DB_ACCESS_KEY).getValue();
-        final ConsistencyLevel clevel;
-        final String selectedConsistency = context.getProperty(AzureCosmosDBUtils.CONSISTENCY).getValue();
-
-        switch(selectedConsistency) {
-            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
-                clevel =  ConsistencyLevel.STRONG;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_CONSISTENT_PREFIX:
-                clevel = ConsistencyLevel.CONSISTENT_PREFIX;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_SESSION:
-                clevel = ConsistencyLevel.SESSION;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_BOUNDED_STALENESS:
-                clevel = ConsistencyLevel.BOUNDED_STALENESS;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_EVENTUAL:
-                clevel = ConsistencyLevel.EVENTUAL;
-                break;
-            default:
-                clevel = ConsistencyLevel.SESSION;
-        }
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();
 
-        if (this.cosmosClient != null) {
+        if (getCosmosClient() != null) {
             onStopped();
         }
-        consistencyLevel = clevel.toString();
-        createCosmosClient(uri, accessKey, clevel);
+        createCosmosClient(
+                getURI(),
+                getAccessKey(),
+                getConsistencyLevel()
+        );

Review comment:
       ```suggestion
           createCosmosClient(getURI(), getAccessKey(), getConsistencyLevel());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosDBClientService.java
##########
@@ -21,91 +21,45 @@
 import java.util.Collection;
 import java.util.List;
 
-import com.azure.cosmos.ConsistencyLevel;
-import com.azure.cosmos.CosmosClient;
-import com.azure.cosmos.CosmosClientBuilder;
-import com.azure.cosmos.CosmosException;
-
 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.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
-import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
 import org.apache.nifi.util.StringUtils;
 
 @Tags({"azure", "cosmos", "document", "service"})
 @CapabilityDescription(
         "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
         " and provides access to that connection to other Cosmos DB-related components."
 )
-public class AzureCosmosDBClientService extends AbstractControllerService implements AzureCosmosDBConnectionService {
+public class AzureCosmosDBClientService
+        extends AbstractCosmosDBClientService {
     private String uri;
     private String accessKey;
     private String consistencyLevel;
-    private CosmosClient cosmosClient;
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) {
         this.uri = context.getProperty(AzureCosmosDBUtils.URI).getValue();
         this.accessKey = context.getProperty(AzureCosmosDBUtils.DB_ACCESS_KEY).getValue();
-        final ConsistencyLevel clevel;
-        final String selectedConsistency = context.getProperty(AzureCosmosDBUtils.CONSISTENCY).getValue();
-
-        switch(selectedConsistency) {
-            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
-                clevel =  ConsistencyLevel.STRONG;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_CONSISTENT_PREFIX:
-                clevel = ConsistencyLevel.CONSISTENT_PREFIX;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_SESSION:
-                clevel = ConsistencyLevel.SESSION;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_BOUNDED_STALENESS:
-                clevel = ConsistencyLevel.BOUNDED_STALENESS;
-                break;
-            case AzureCosmosDBUtils.CONSISTENCY_EVENTUAL:
-                clevel = ConsistencyLevel.EVENTUAL;
-                break;
-            default:
-                clevel = ConsistencyLevel.SESSION;
-        }
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();

Review comment:
       ```suggestion
           this.consistencyLevel = context.getProperty(AzureCosmosDBUtils.CONSISTENCY).getValue();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/services/azure/storage/TestADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.storage;
+
+import java.util.HashMap;
+
+import com.azure.security.keyvault.secrets.SecretClient;
+
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultClientService;
+import org.apache.nifi.util.NoOpProcessor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+class MockKeyVaultConnectionService extends AzureKeyVaultClientService {
+
+    private static final String ACCOUNT_NAME_SECRET_NAME = "AccountNameSecret";
+    private static final String ACCOUNT_KEY_SECRET_NAME = "AccountKeySecret";
+    private static final String SAS_TOKEN_SECRET_NAME = "SasTokenSecret";
+    private static final String END_POINT_SUFFIX_SECRET_NAME = "end.point.suffix";
+
+    private String ACCOUNT_NAME_SECRET_VALUE = "AccountNameSecretValue";
+    private String ACCOUNT_KEY_SECRET_VALUE = "AccountKeySecretValue";
+    private String SAS_TOKEN_SECRET_VALUE = "SasTokenSecretValue";
+    private String END_POINT_SUFFIX_SECRET_VALUE = "end.point.suffix";
+
+    private String MOCK_INVALID_SECRET_VALUE = "INVALID_VALUE";
+
+    @Override
+    public SecretClient getKeyVaultSecretClient(){
+        return mock(SecretClient.class);
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        switch(secretName) {
+            case ACCOUNT_NAME_SECRET_NAME:
+                return ACCOUNT_NAME_SECRET_VALUE;
+            case ACCOUNT_KEY_SECRET_NAME:
+                return ACCOUNT_KEY_SECRET_VALUE;
+            case SAS_TOKEN_SECRET_NAME:
+                return SAS_TOKEN_SECRET_VALUE;
+            case END_POINT_SUFFIX_SECRET_NAME:
+                return END_POINT_SUFFIX_SECRET_VALUE;
+            default:
+               return MOCK_INVALID_SECRET_VALUE;
+        }
+    }
+}
+
+public class TestADLSSecureCredentialsControllerService {
+
+    public static final String CREDENTIALS_SERVICE_IDENTIFIER = "secure-credentials-service";
+
+    private static final String ACCOUNT_NAME_SECRET = "AccountNameSecret";
+    private static final String ACCOUNT_KEY_SECRET = "AccountKeySecret";
+    private static final String SAS_TOKEN_SECRET = "SasTokenSecret";
+    private static final String END_POINT_SUFFIX_SECRET = "end.point.suffix";
+
+    private String ACCOUNT_NAME_SECRET_VALUE = "AccountNameSecretValue";
+    private String ACCOUNT_KEY_SECRET_VALUE = "AccountKeySecretValue";
+    private String SAS_TOKEN_SECRET_VALUE = "SasTokenSecretValue";
+    private String END_POINT_SUFFIX_SECRET_VALUE = "end.point.suffix";
+
+    private TestRunner runner;
+    private ADLSSecureCredentialsControllerService credentialsService;
+
+    @Before
+    public void setUp() throws InitializationException {
+        runner = TestRunners.newTestRunner(NoOpProcessor.class);
+        credentialsService = new ADLSSecureCredentialsControllerService();
+        runner.addControllerService(CREDENTIALS_SERVICE_IDENTIFIER, credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseAccountNameMissing() {
+        configureAccountKey();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseNoCredentialsIsSet() {
+        configureAccountName();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidBecauseBothAccountKeyAndSasTokenSpecified() {
+        configureAccountName();
+        configureAccountKey();
+        configureSasToken();
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidWithEmptyEndpointSuffix() {
+        configureAccountName();
+        configureAccountKey();
+
+        runner.setProperty(credentialsService, AzureStorageUtils.ADLS_ENDPOINT_SUFFIX, "");
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testNotValidWithWhitespaceEndpointSuffix() {
+        configureAccountName();
+        configureAccountKey();
+        runner.setProperty(credentialsService, AzureStorageUtils.ADLS_ENDPOINT_SUFFIX, " ");
+
+        runner.assertNotValid(credentialsService);
+    }
+
+    @Test
+    public void testValidWithAccountNameAndAccountKey() throws InitializationException {
+        configureAccountName();
+        configureAccountKey();
+        configureKeyVaultConnectionService();
+
+        runner.assertValid(credentialsService);
+    }
+
+    @Test
+    public void testValidWithAccountNameAndSasToken() throws InitializationException {
+        configureAccountName();
+        configureSasToken();
+        configureKeyVaultConnectionService();
+
+        runner.assertValid(credentialsService);
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithAccountKey() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureAccountKey();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(ACCOUNT_NAME_SECRET_VALUE, actual.getAccountName());
+        assertEquals(ACCOUNT_KEY_SECRET_VALUE, actual.getAccountKey());
+        assertNull(actual.getSasToken());
+        assertFalse(actual.getUseManagedIdentity());
+        assertNotNull(actual.getEndpointSuffix());
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithSasToken() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureSasToken();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(ACCOUNT_NAME_SECRET_VALUE, actual.getAccountName());
+        assertEquals(SAS_TOKEN_SECRET_VALUE, actual.getSasToken());
+        assertNull(actual.getAccountKey());
+        assertFalse(actual.getUseManagedIdentity());
+        assertNotNull(actual.getEndpointSuffix());
+    }
+
+    @Test
+    public void testGetCredentialsDetailsWithSetEndpointSuffix() throws Exception {
+        // GIVEN
+        configureAccountName();
+        configureAccountKey();
+        configureEndpointSuffix();
+        configureKeyVaultConnectionService();
+
+        runner.enableControllerService(credentialsService);
+
+        // WHEN
+        ADLSCredentialsDetails actual = credentialsService.getCredentialsDetails(new HashMap<>());
+
+        // THEN
+        assertEquals(END_POINT_SUFFIX_SECRET, actual.getEndpointSuffix());
+    }
+

Review comment:
       What happens when the client service returns an empty or blank string? I think we should add test cases for those.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {

Review comment:
       ```suggestion
   public abstract class AbstractCosmosDBClientService extends AbstractControllerService implements AzureCosmosDBConnectionService {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    private CosmosClient cosmosClient;
+
+    @Override
+    public CosmosClient getCosmosClient() {
+        return this.cosmosClient;
+    }
+
+    protected void setCosmosClient(CosmosClient client) {
+        this.cosmosClient = client;
+    }
+
+    @OnStopped
+    public final void onStopped() {
+        if (this.cosmosClient != null) {
+            try {
+                cosmosClient.close();
+            } catch(CosmosException e) {
+                getLogger().error("Closing CosmosClient Failed: " + e.getMessage(), e);
+            } finally {
+                this.cosmosClient = null;
+            }
+        }
+    }
+
+    protected void createCosmosClient(final String uri, final String accessKey, final String selectedConsistency){

Review comment:
       Can we rename this to `initCosmosClient()` for clarity -- it doesn't create the client so much as mutate the internal state and initialize it.
   ```suggestion
       protected void initCosmosClient(final String uri, final String accessKey, final String selectedConsistency) {
   ```




-- 
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594535381



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })

Review comment:
       Not sure, but this is same as https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageCredentialsControllerService.java#L44




----------------------------------------------------------------
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.

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



[GitHub] [nifi] github-actions[bot] closed pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #4863:
URL: https://github.com/apache/nifi/pull/4863


   


-- 
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



[GitHub] [nifi] sjyang18 commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sjyang18 commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-790849775


   let you know that there is a conflicting file. Please resolve it. And, adding PR # in commit message is the convention in NiFi. 


----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r681450117



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String accountKey = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        final String accountName = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        final String sasToken = validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if (StringUtils.isBlank(accountName)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()
+                            + " is required")
+                    .build());
+        }
+
+        if (StringUtils.isBlank(accountKey) && StringUtils.isBlank(sasToken)) {
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("either "
+                            + AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName()
+                            + " or "
+                            + AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName()
+                            + " must be defined")
+                    .build());
+        }
+        return results;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public AzureStorageCredentialsDetails getStorageCredentialsDetails(Map<String, String> attributes) {
+        final String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).evaluateAttributeExpressions(attributes).getValue();
+        final String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+        ).evaluateAttributeExpressions(attributes).getValue();
+        final String storageSuffix = context.getProperty(
+                AzureStorageUtils.ENDPOINT_SUFFIX
+        ).evaluateAttributeExpressions(attributes).getValue();;
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+
+        if (StringUtils.isBlank(accountNameSecret)) {
+            throw new IllegalArgumentException(String.format(
+                    "'%s' must not be empty.", AzureStorageUtils.ACCOUNT_NAME_SECRET.getDisplayName()));
+        }
+
+        String accountNameValue = keyVaultClientService.getSecret(accountNameSecret);
+
+        StorageCredentials storageCredentials;
+
+        if (StringUtils.isNotBlank(accountKeySecret)) {
+            String accountKeyValue = keyVaultClientService.getSecret(accountKeySecret);

Review comment:
       There would be an error while creating the client




-- 
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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594774361



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();

Review comment:
       cleaned




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-1009520308


   > @exceptionfactory I think when this started the sensitive parameter provider work hadn't been sketched out yet. I think it is a good intermediate solution.
   
   Thanks @jfrazee! I defer to you on the details and would be glad to take another look at this PR once it looks good to you.


-- 
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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594774241



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {
+        return this.useManagedIdentity;
+    }
+
+    @Override
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)

Review comment:
       I am not sure regarding this, are you asking to refactor so that these service principal details could also be fetched from a keyvault. If it is about supporting the client certificate I would want to make that change once NIFI-8277 is done.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] github-actions[bot] commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-1001085890


   We're marking this PR as stale due to lack of updates in the past few months. If after another couple of weeks the stale label has not been removed this PR will be closed. This stale marker and eventual auto close does not indicate a judgement of the PR just lack of reviewer bandwidth and helps us keep the PR queue more manageable.  If you would like this PR re-opened you can do so and a committer can remove the stale tag.  Or you can open a new PR.  Try to help review other PRs to increase PR review bandwidth which in turn helps yours.


-- 
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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r587852235



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

Review comment:
       fixed




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594774750



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSCredentialsControllerService.java
##########
@@ -221,4 +211,4 @@ public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attribut
             setBuilderValue.accept(credentialsBuilder, value);
         }
     }
-}
+}

Review comment:
       Fixed




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r596016120



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {
+        return this.useManagedIdentity;
+    }
+
+    @Override
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)

Review comment:
       This method, and the other existing approach to determining which type of client credential to use share some of the same logic, so it would be helpful to reduce the duplication down the road.  It sounds like it is better to wait until NIFI-8277 to look at refactoring.  




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594776636



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();
+
+    String getSecret(String secretName);
+
+    String getSecretFromKeyVault(String secretName);

Review comment:
       cleaned




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594774241



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {
+        return this.useManagedIdentity;
+    }
+
+    @Override
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)

Review comment:
       I am not sure regarding this, are you asking to refactor so that these service principal details could also be fetched from a keyvault

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+    private ComponentLog logger;

Review comment:
       cleaned




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594782299



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+            AzureStorageUtils.ACCOUNT_NAME_SECRET,
+            AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+            AzureStorageUtils.ACCOUNT_KEY_SECRET,
+            AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+    ));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        boolean accountKeySet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue());
+        boolean sasTokenSet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue());
+
+        if (!onlyOneSet(accountKeySet, sasTokenSet)) {
+            StringJoiner options = new StringJoiner(", ")
+                    .add(AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName())
+                    .add(AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName());
+
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("one and only one of [" + options + "] should be set")
+                    .build());
+        }
+

Review comment:
       maybe I misunderstood, but keyvault does not allow blank values. Let me know if you wanted me to validate something else here.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-1009491307


   The concept of supporting integration with Azure KeyVault is great, but the implementation in this PR raises some questions.  The ideal implementation would support Azure KeyVault at a lower level, as opposed to an explicit Controller Service, but perhaps this approach might be a reasonable intermediate solution.  That being said, it would be helpful to narrow the scope of this PR to the Azure KeyVault service and implementation itself, and then look at integrating the service in a separate PR.


-- 
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



[GitHub] [nifi] joewitt commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-1009477495


   Seems like this had a lot of conversation and effort and should continue.  @sushilkm Any intent to keep this one going?


-- 
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



[GitHub] [nifi] sjyang18 commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sjyang18 commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-790911461


   compilation error needs fixing.


----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594776427



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        this.uriSecret = context.getProperty(URI_SECRET).getValue();
+        this.accessKeySecret = context.getProperty(DB_ACCESS_KEY_SECRET).getValue();
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();
+        this.keyVaultClientService = context.getProperty(
+                KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        createCosmosClient(
+                getURI(),
+                getAccessKey(),
+                getConsistencyLevel()
+        );
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();

Review comment:
       fixed




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sjyang18 commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sjyang18 commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r587740543



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

Review comment:
       NON_BLANK_VALIDATOR?
   




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r697378737



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSCredentialsControllerService.java
##########
@@ -183,35 +171,35 @@ public void onEnabled(ConfigurationContext context) {
     public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attributes) {
         ADLSCredentialsDetails.Builder credentialsBuilder = ADLSCredentialsDetails.Builder.newBuilder();
 
-        setValue(credentialsBuilder, ACCOUNT_NAME, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setAccountName, attributes);
-        setValue(credentialsBuilder, AzureStorageUtils.ACCOUNT_KEY, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setAccountKey, attributes);
-        setValue(credentialsBuilder, AzureStorageUtils.PROP_SAS_TOKEN, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setSasToken, attributes);
-        setValue(credentialsBuilder, ENDPOINT_SUFFIX, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setEndpointSuffix, attributes);
-        setValue(credentialsBuilder, USE_MANAGED_IDENTITY, PropertyValue::asBoolean, ADLSCredentialsDetails.Builder::setUseManagedIdentity, attributes);
-        setValue(credentialsBuilder, SERVICE_PRINCIPAL_TENANT_ID, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalTenantId, attributes);
-        setValue(credentialsBuilder, SERVICE_PRINCIPAL_CLIENT_ID, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalClientId, attributes);
-        setValue(credentialsBuilder, SERVICE_PRINCIPAL_CLIENT_SECRET, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalClientSecret, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, ACCOUNT_NAME, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setAccountName, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, AzureStorageUtils.ACCOUNT_KEY, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setAccountKey, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, AzureStorageUtils.PROP_SAS_TOKEN, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setSasToken, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, AzureStorageUtils.ADLS_ENDPOINT_SUFFIX, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setEndpointSuffix, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, USE_MANAGED_IDENTITY, PropertyValue::asBoolean, ADLSCredentialsDetails.Builder::setUseManagedIdentity, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, SERVICE_PRINCIPAL_TENANT_ID, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalTenantId, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, SERVICE_PRINCIPAL_CLIENT_ID, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalClientId, attributes);
+        AzureStorageUtils.setValue(context, credentialsBuilder, SERVICE_PRINCIPAL_CLIENT_SECRET, PropertyValue::getValue, ADLSCredentialsDetails.Builder::setServicePrincipalClientSecret, attributes);
 
         return credentialsBuilder.build();
     }
 
-    private <T> void setValue(
-            ADLSCredentialsDetails.Builder credentialsBuilder,
-            PropertyDescriptor propertyDescriptor, Function<PropertyValue, T> getPropertyValue,
-            BiConsumer<ADLSCredentialsDetails.Builder, T> setBuilderValue, Map<String, String> attributes
-    ) {
-        PropertyValue property = context.getProperty(propertyDescriptor);
-
-        if (property.isSet()) {
-            if (propertyDescriptor.isExpressionLanguageSupported()) {
-                if (propertyDescriptor.getExpressionLanguageScope() == ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) {
-                    property = property.evaluateAttributeExpressions(attributes);
-                } else {
-                    property = property.evaluateAttributeExpressions();
-                }
-            }
-            T value = getPropertyValue.apply(property);
-            setBuilderValue.accept(credentialsBuilder, value);
-        }
-    }
+//    private <T> void setValue(
+//            ADLSCredentialsDetails.Builder credentialsBuilder,
+//            PropertyDescriptor propertyDescriptor, Function<PropertyValue, T> getPropertyValue,
+//            BiConsumer<ADLSCredentialsDetails.Builder, T> setBuilderValue, Map<String, String> attributes
+//    ) {
+//        PropertyValue property = context.getProperty(propertyDescriptor);
+//
+//        if (property.isSet()) {
+//            if (propertyDescriptor.isExpressionLanguageSupported()) {
+//                if (propertyDescriptor.getExpressionLanguageScope() == ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) {
+//                    property = property.evaluateAttributeExpressions(attributes);
+//                } else {
+//                    property = property.evaluateAttributeExpressions();
+//                }
+//            }
+//            T value = getPropertyValue.apply(property);
+//            setBuilderValue.accept(credentialsBuilder, value);
+//        }
+//    }

Review comment:
       Is there a reason for leaving this code commented out? If it is not used, it should be removed.




-- 
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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594774841



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;

Review comment:
       fixed




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594773232



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled

Review comment:
       Fixed




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sjyang18 edited a comment on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sjyang18 edited a comment on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-790911461


   compilation failures needs fixing.


----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r590388525



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)

Review comment:
       With the introduction of sensitive Parameters, it is not good practice to support expression language for sensitive properties.  Inserting sensitive property values in FlowFile attributes does not provide encryption by default when storing provenance records.  Although other Azure components indicated support for expression language in FlowFile attributes, it was not actually implemented, so PR #4843 removed support.  Is there a particular reason for support expression language in these sensitive properties?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;
+
+    @OnStopped
+    public final void onStopped() {
+        if (this.cosmosClient != null) {
+            try {
+                cosmosClient.close();
+            } catch(CosmosException e) {
+                getLogger().error("Closing CosmosClient Failed: " + e.getMessage(), e);
+            } finally {
+                this.cosmosClient = null;
+            }
+        }
+    }
+
+    protected void createCosmosClient(final String uri, final String accessKey, final String selectedConsistency){
+        final ConsistencyLevel cLevel;
+
+        switch(selectedConsistency) {
+            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
+                cLevel =  ConsistencyLevel.STRONG;

Review comment:
       It looks like there is an extra space separating the value assignment in this line.
   ```suggestion
                   cLevel = ConsistencyLevel.STRONG;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -143,6 +144,54 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("Storage Account Name Secret Name")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("Storage Account Key Secret Name")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-sas-token")
+            .displayName("SAS Token Secret Name")
+            .description("SAS Token Secret Name")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+
+    public static final PropertyDescriptor ADLS_ENDPOINT_SUFFIX = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AzureStorageUtils.ENDPOINT_SUFFIX)
+            .displayName("Endpoint Suffix")
+            .description(
+                "Storage accounts in public Azure always use a common FQDN suffix. " +
+                    "Override this endpoint suffix with a different suffix in certain circumstances (like Azure Stack or non-public Azure regions).")
+            .required(true)
+            .defaultValue("dfs.core.windows.net")
+            .build();

Review comment:
       Should this property descriptor have some kind of validator, such as the NON_BLANK_VALIDATOR, or is blank valid?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {

Review comment:
       Are the above public get methods necessary?  In general, public methods on Controller Service implementations that do not implement interface methods should be avoided.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+

Review comment:
       It would be helpful to add some basic comments about the purpose and capabilities of this interface.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();
+
+    String getSecret(String secretName);
+
+    String getSecretFromKeyVault(String secretName);

Review comment:
       Is it necessary to have a separate method with basically the same signature as `getSecret`?  The implementation purpose seems to be avoiding checking the cache, but it seems better to simplify the interface and have the implementation drive the usage.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        this.uriSecret = context.getProperty(URI_SECRET).getValue();
+        this.accessKeySecret = context.getProperty(DB_ACCESS_KEY_SECRET).getValue();
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();
+        this.keyVaultClientService = context.getProperty(
+                KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        createCosmosClient(
+                getURI(),
+                getAccessKey(),
+                getConsistencyLevel()
+        );
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(KEYVAULT_CONNECTION_SERVICE);
+        descriptors.add(URI_SECRET);
+        descriptors.add(DB_ACCESS_KEY_SECRET);
+        descriptors.add(AzureCosmosDBUtils.CONSISTENCY);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public String getURI() {
+        if (keyVaultClientService == null) {
+            throw new IllegalArgumentException(String.format(
+                    "Cannot get '%s'.", KEYVAULT_CONNECTION_SERVICE.getDisplayName()));
+        }
+        if (StringUtils.isBlank(uriSecret)) {

Review comment:
       Is it possible for `uriSecret` to be blank in light of the property validation and custom validate implementation?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultClientService.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.keyvault;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.util.StringUtils;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.security.keyvault.secrets.SecretClient;
+import com.azure.security.keyvault.secrets.SecretClientBuilder;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({"azure", "keyvault", "credential", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Azure Key Vault" +
+                " and provides access to that connection to Azure Key Vault components."
+)
+public class AzureKeyVaultClientService
+        extends AbstractControllerService
+        implements AzureKeyVaultConnectionService {
+
+    private String keyVaultName;
+    private String servicePrincipalClientID;
+    private String servicePrincipalClientSecret;
+    private String tenantID;
+    private String endPointSuffix;
+    private Boolean useManagedIdentity;
+    private SecretClient keyVaultSecretClient;
+    private ComponentLog logger;
+    private LoadingCache<String, String> secretCache;
+
+    public String getKeyVaultName() {
+        return this.keyVaultName;
+    }
+
+    public String getServicePrincipalClientID() {
+        return this.servicePrincipalClientID;
+    }
+
+    public String getServicePrincipalClientSecret() {
+        return this.servicePrincipalClientSecret;
+    }
+
+    public String getTenantID() {
+        return this.tenantID;
+    }
+
+    public String getEndPointSuffix() {
+        return this.endPointSuffix;
+    }
+
+    public Boolean getUseManagedIdentity() {
+        return this.useManagedIdentity;
+    }
+
+    @Override
+    public SecretClient getKeyVaultSecretClient() {
+        return this.keyVaultSecretClient;
+    }
+
+    @Override
+    public String getSecretFromKeyVault(String secretName) {
+        return this.keyVaultSecretClient.getSecret(secretName).getValue();
+    }
+
+    @Override
+    public String getSecret(String secretName) {
+        if (secretCache != null) {
+            try {
+                return secretCache.get(secretName);
+            } catch (final Exception e) {
+                logger.error("Failed to get secret '"+ secretName +"' from cache", e);
+            }
+        }
+        return getSecretFromKeyVault(secretName);
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        logger = getLogger();
+        this.keyVaultName = context.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        this.servicePrincipalClientID = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        this.servicePrincipalClientSecret = context.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        this.tenantID = context.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        this.endPointSuffix = context.getProperty(
+                AzureKeyVaultUtils.ENDPOINT_SUFFIX).getValue();
+        this.useManagedIdentity = context.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        createKeyVaultSecretClient();
+
+        final Integer cacheSize = context.getProperty(AzureKeyVaultUtils.CACHE_SIZE).asInteger();
+        final Long cacheTTL = context.getProperty(
+                AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE
+        ).asTimePeriod(TimeUnit.SECONDS);
+
+        if (cacheSize > 0) {
+            CacheBuilder<Object, Object> cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
+            if (cacheTTL > 0) {
+                cacheBuilder = cacheBuilder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS);
+            }
+
+            logger.info(String.format(
+                    "Secret cache enabled with cacheSize: %d and cacheTTL: %d secs",
+                    cacheSize, cacheTTL));
+            secretCache = cacheBuilder.build(
+                    new CacheLoader<String, String>() {
+                        @Override
+                        public String load(String secretName) throws Exception {
+                            return getSecretFromKeyVault(secretName);
+                        }
+                    });
+        } else {
+            secretCache = null;
+            logger.info("Secret cache disabled because cache size is set to 0");
+        }
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();
+
+    static {
+        descriptors.add(AzureKeyVaultUtils.KEYVAULT_NAME);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_ID);
+        descriptors.add(AzureKeyVaultUtils.SP_CLIENT_SECRET);
+        descriptors.add(AzureKeyVaultUtils.TENANT_ID);
+        descriptors.add(AzureKeyVaultUtils.ENDPOINT_SUFFIX);
+        descriptors.add(AzureKeyVaultUtils.USE_MANAGED_IDENTITY);
+        descriptors.add(AzureKeyVaultUtils.CACHE_SIZE);
+        descriptors.add(AzureKeyVaultUtils.CACHE_TTL_AFTER_WRITE);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        final String keyVaultName = validationContext.getProperty(
+                AzureKeyVaultUtils.KEYVAULT_NAME).getValue();
+        final String clientID = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_ID).getValue();
+        final String clientSecret = validationContext.getProperty(
+                AzureKeyVaultUtils.SP_CLIENT_SECRET).getValue();
+        final String tenantID = validationContext.getProperty(
+                AzureKeyVaultUtils.TENANT_ID).getValue();
+        final Boolean useManagedIdentity = validationContext.getProperty(
+                AzureKeyVaultUtils.USE_MANAGED_IDENTITY).asBoolean();
+
+        if (StringUtils.isBlank(keyVaultName)) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation(AzureKeyVaultUtils.KEYVAULT_NAME.getDisplayName() +" is required")
+                    .build());
+        } else if (useManagedIdentity && (StringUtils.isNotBlank(clientID)
+                || StringUtils.isNotBlank(clientSecret))) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("if " + AzureKeyVaultUtils.USE_MANAGED_IDENTITY.getDisplayName()
+                            + " is used then " + AzureKeyVaultUtils.SP_CLIENT_ID.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.SP_CLIENT_SECRET.getDisplayName()
+                            + ", " + AzureKeyVaultUtils.TENANT_ID.getDisplayName()
+                            + " should be blank.")
+                    .build());
+        } else if (!useManagedIdentity && (StringUtils.isBlank(clientID)

Review comment:
       PR #4843 introduced support for Service Principals and NIFI-8277 proposes support for Client Certificate credentials.  This method and `createKeyValueSecretClient()` would benefit from some refactoring along with the existing credential handling to avoid duplication and ensure that that new credential implementations would be supported.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AbstractCosmosDBClientService.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.cosmos.document;
+
+import com.azure.cosmos.ConsistencyLevel;
+import com.azure.cosmos.CosmosClient;
+import com.azure.cosmos.CosmosClientBuilder;
+import com.azure.cosmos.CosmosException;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+
+@Tags({"azure", "cosmos", "document", "service"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public abstract class AbstractCosmosDBClientService
+        extends AbstractControllerService
+        implements AzureCosmosDBConnectionService {
+
+    protected CosmosClient cosmosClient;
+
+    @OnStopped
+    public final void onStopped() {
+        if (this.cosmosClient != null) {
+            try {
+                cosmosClient.close();
+            } catch(CosmosException e) {
+                getLogger().error("Closing CosmosClient Failed: " + e.getMessage(), e);
+            } finally {
+                this.cosmosClient = null;
+            }
+        }
+    }
+
+    protected void createCosmosClient(final String uri, final String accessKey, final String selectedConsistency){
+        final ConsistencyLevel cLevel;
+
+        switch(selectedConsistency) {
+            case AzureCosmosDBUtils.CONSISTENCY_STRONG:
+                cLevel =  ConsistencyLevel.STRONG;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_CONSISTENT_PREFIX:
+                cLevel = ConsistencyLevel.CONSISTENT_PREFIX;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_BOUNDED_STALENESS:
+                cLevel = ConsistencyLevel.BOUNDED_STALENESS;
+                break;
+            case AzureCosmosDBUtils.CONSISTENCY_EVENTUAL:
+                cLevel = ConsistencyLevel.EVENTUAL;
+                break;
+            default:
+                cLevel = ConsistencyLevel.SESSION;
+        }
+        this.cosmosClient = new CosmosClientBuilder()
+                .endpoint(uri)
+                .key(accessKey)
+                .consistencyLevel(cLevel)
+                .buildClient();
+    }
+
+    @Override
+    public CosmosClient getCosmosClient() {
+        return this.cosmosClient;
+    }
+
+    public void setCosmosClient(CosmosClient client) {

Review comment:
       Is this method used only for testing?  If so, it would be better to override the `cosmosClient` directly instead of having a public set method.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled

Review comment:
       It would be helpful to add a newline above this line for readability.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageSecureCredentialsControllerService.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.storage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.microsoft.azure.storage.StorageCredentials;
+import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+/**
+ * Implementation of AbstractControllerService interface
+ *
+ * @see AbstractControllerService
+ */
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob", "queue", "credentials" })
+@CapabilityDescription("Defines credentials for Azure Storage processors. " +
+        "Uses Account Name with Account Key or Account Name with SAS Token.")
+public class AzureStorageSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements AzureStorageCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections
+            .unmodifiableList(Arrays.asList(
+                    AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+                    AzureStorageUtils.ACCOUNT_NAME_SECRET,
+                    AzureStorageUtils.ACCOUNT_KEY_SECRET,
+                    AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET,
+                    AzureStorageUtils.ENDPOINT_SUFFIX));
+
+    private ConfigurationContext context;
+    private ComponentLog logger;

Review comment:
       Is there a reason for assigning logger as opposed to calling `getLogger()` when needed?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/cosmos/document/AzureCosmosSecureDBClientService.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.cosmos.document;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+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.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.cosmos.document.AzureCosmosDBUtils;
+import org.apache.nifi.services.azure.cosmos.AzureCosmosDBConnectionService;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+import org.apache.nifi.util.StringUtils;
+
+@Tags({"azure", "cosmos", "document", "service", "secure"})
+@CapabilityDescription(
+        "Provides a controller service that configures a connection to Cosmos DB (Core SQL API) " +
+                " and provides access to that connection to other Cosmos DB-related components."
+)
+public class AzureCosmosSecureDBClientService
+        extends AbstractCosmosDBClientService
+        implements AzureCosmosDBConnectionService {
+
+    private String uriSecret;
+    private String accessKeySecret;
+    private String consistencyLevel;
+    private AzureKeyVaultConnectionService keyVaultClientService;
+
+    public static final PropertyDescriptor URI_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-uri-secret")
+            .displayName("Cosmos DB URI Secret Name")
+            .description("Cosmos DB URI Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.URI_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor DB_ACCESS_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("azure-cosmos-db-key-secret")
+            .displayName("Cosmos DB Access Key Secret Name")
+            .description("Cosmos DB Access Key Secret Name")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    static final PropertyDescriptor KEYVAULT_CONNECTION_SERVICE = new PropertyDescriptor.Builder()
+            .name("azure-keyvault-connection-service")
+            .displayName("KeyVault Connection Service")
+            .description("If configured, the controller service used to obtain the secrets from KeyVault")
+            .required(true)
+            .identifiesControllerService(AzureKeyVaultConnectionService.class)
+            .build();
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        this.uriSecret = context.getProperty(URI_SECRET).getValue();
+        this.accessKeySecret = context.getProperty(DB_ACCESS_KEY_SECRET).getValue();
+        this.consistencyLevel = context.getProperty(
+                AzureCosmosDBUtils.CONSISTENCY).getValue();
+        this.keyVaultClientService = context.getProperty(
+                KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+
+        createCosmosClient(
+                getURI(),
+                getAccessKey(),
+                getConsistencyLevel()
+        );
+    }
+
+    static List<PropertyDescriptor> descriptors = new ArrayList<>();

Review comment:
       Following standard conventions, static variables and initializers should be declared before non-static methods.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();

Review comment:
       Is it necessary to expose the SecretClient as a interface method?  The interface could be simplified to something more generic if it is not necessary to make the SecretClient available to consumers.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] jfrazee commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
jfrazee commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r593342218



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();

Review comment:
       I think I'd go stronger and say this really must use an interface internal to NiFi. We've created dependency upgrade issues previously by exposing the SDK types.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] sushilkm commented on pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#issuecomment-891438441


   > Hey @sushilkm, thanks for this. I've gone through all the code.
   > 
   > * I think the changes for the Cosmos DB CS are good.
   > * I made some style suggestions around descriptions for the various property descriptors. In most cases it's to make the description a noun and to shorten things up.
   > * The current formatting is pretty aggressive on line lengths so I suggested some changes. It's ok to go over 80 but needs to be less than 200 to pass the checkstyle. I'd say you're ok not breaking lines until you're around 110 or 120 or so.
   > * I do have a question about why these are `*SecureCredentialsControllerService` and not `*KeyVaultCredentialsControllerService`. Is there an intended difference?
   
   Thank you @jfrazee for reviewing this.
   
   As suggested I have made changes in the PR.
   
   - Regarding the checkstyle thing, the tests works fine so I was not sure how should I further work on finding the issues.
   - I have updated the name from `*SecureCredentialsControllerService` to `*KeyVaultCredentialsControllerService`, because it made sense and there was no intended difference for the name.
   Please have a look at this again.
   
   Thank you


-- 
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



[GitHub] [nifi] sushilkm commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
sushilkm commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r594780896



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/ADLSSecureCredentialsControllerService.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.storage;
+
+import org.apache.commons.lang3.StringUtils;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.keyvault.AzureKeyVaultConnectionService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * Provides secure credentials details for ADLS
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "adls", "credentials", "secure"})
+@CapabilityDescription("Defines credentials for ADLS processors.")
+public class ADLSSecureCredentialsControllerService
+        extends AbstractControllerService
+        implements ADLSCredentialsService {
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE,
+            AzureStorageUtils.ACCOUNT_NAME_SECRET,
+            AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+            AzureStorageUtils.ACCOUNT_KEY_SECRET,
+            AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET
+    ));
+
+    private ConfigurationContext context;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+
+        boolean accountKeySet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue());
+        boolean sasTokenSet = StringUtils.isNotBlank(validationContext.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue());
+
+        if (!onlyOneSet(accountKeySet, sasTokenSet)) {
+            StringJoiner options = new StringJoiner(", ")
+                    .add(AzureStorageUtils.ACCOUNT_KEY_SECRET.getDisplayName())
+                    .add(AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET.getDisplayName());
+
+            results.add(new ValidationResult.Builder().subject(this.getClass().getSimpleName())
+                    .valid(false)
+                    .explanation("one and only one of [" + options + "] should be set")
+                    .build());
+        }
+
+        return results;
+    }
+
+    private boolean onlyOneSet(Boolean... checks) {
+        long nrOfSet = Arrays.stream(checks)
+                .filter(check -> check)
+                .count();
+
+        return nrOfSet == 1;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        this.context = context;
+    }
+
+    @Override
+    public ADLSCredentialsDetails getCredentialsDetails(Map<String, String> attributes) {
+        ADLSCredentialsDetails.Builder credentialsBuilder = ADLSCredentialsDetails.Builder.newBuilder();
+
+        setValue(credentialsBuilder,
+                AzureStorageUtils.ADLS_ENDPOINT_SUFFIX,
+                PropertyValue::getValue,
+                ADLSCredentialsDetails.Builder::setEndpointSuffix);
+
+        final AzureKeyVaultConnectionService keyVaultClientService = context.getProperty(
+                AzureStorageUtils.KEYVAULT_CONNECTION_SERVICE
+        ).asControllerService(AzureKeyVaultConnectionService.class);
+        String accountNameSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_NAME_SECRET).getValue();
+        String accountKeySecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_KEY_SECRET).getValue();
+        String sasTokenSecret = context.getProperty(
+                AzureStorageUtils.ACCOUNT_SAS_TOKEN_SECRET).getValue();
+
+        if(StringUtils.isNotBlank(accountNameSecret)) {
+            credentialsBuilder.setAccountName(
+                    keyVaultClientService.getSecret(accountNameSecret));
+        }
+        if(StringUtils.isNotBlank(accountKeySecret)) {
+            credentialsBuilder.setAccountKey(
+                    keyVaultClientService.getSecret(accountKeySecret));

Review comment:
       Keyvault does not allows empty secret, in case of a non-existent secret requested there would be an error/exception from keyvault sdk




----------------------------------------------------------------
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.

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r593586097



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-services-api/src/main/java/org/apache/nifi/services/azure/keyvault/AzureKeyVaultConnectionService.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.keyvault;
+
+import org.apache.nifi.controller.ControllerService;
+import com.azure.security.keyvault.secrets.SecretClient;
+
+public interface AzureKeyVaultConnectionService extends ControllerService {
+
+    SecretClient getKeyVaultSecretClient();

Review comment:
       Very good point @jfrazee, I agree this needs to be changed.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] pvillard31 commented on a change in pull request #4863: NIFI-8279: Adds secure controller service using keyvault secrets and tests

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on a change in pull request #4863:
URL: https://github.com/apache/nifi/pull/4863#discussion_r697334747



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/utils/AzureStorageUtils.java
##########
@@ -156,6 +164,57 @@
             .required(false)
             .build();
 
+    public static final PropertyDescriptor ACCOUNT_NAME_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-name-secret")
+            .displayName("Storage Account Name Secret Name")
+            .description("The name of the Azure Key Vault secret containing the Azure Storage account " +
+                    " name. The Controller Service will use this name to get the value of the secret from " +
+                    " Key Vault.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_KEY_SECRET = new PropertyDescriptor.Builder()
+            .name("storage-account-key-secret")
+            .displayName("Storage Account Key Secret Name")
+            .description("The name of the Azure Key Vault secret containing the Azure Storage account " +
+                    " access key. The Controller Service will use this name to get the value of the secret from " +
+                    " Key Vault.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(false)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor ACCOUNT_SAS_TOKEN_SECRET = new PropertyDescriptor.Builder()
+            .name("-secret")

Review comment:
       typo?




-- 
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