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/10/26 17:43:17 UTC

[GitHub] [nifi] turcsanyip opened a new pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

turcsanyip opened a new pull request #5486:
URL: https://github.com/apache/nifi/pull/5486


   …library v12 for Java
   
   https://issues.apache.org/jira/browse/NIFI-9338
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] 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.

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

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



[GitHub] [nifi] pvillard31 commented on pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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


   Hey @jfrazee - I played with the processors added by this pull request and it was working as expected. Are you fine with the changes made by @turcsanyip following your initial review? Did you get the chance to make more tests?


-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/DeleteAzureBlobStorage_v12.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.processors.azure.storage;
+
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.DeleteSnapshotsOptionType;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AbstractAzureBlobProcessor_v12;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob" })
+@SeeAlso({ ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, PutAzureBlobStorage_v12.class})
+@CapabilityDescription("Deletes the specified blob from Azure Blob Storage. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+public class DeleteAzureBlobStorage_v12 extends AbstractAzureBlobProcessor_v12 {
+
+    public static final AllowableValue DELETE_SNAPSHOTS_NONE = new AllowableValue("NONE", "None", "Delete the blob only.");
+
+    public static final AllowableValue DELETE_SNAPSHOTS_ALSO = new AllowableValue(DeleteSnapshotsOptionType.INCLUDE.name(), "Include Snapshots", "Delete the blob and its snapshots.");
+
+    public static final AllowableValue DELETE_SNAPSHOTS_ONLY = new AllowableValue(DeleteSnapshotsOptionType.ONLY.name(), "Delete Snapshots Only", "Delete only the blob's snapshots.");
+
+    public static final PropertyDescriptor DELETE_SNAPSHOTS_OPTION = new PropertyDescriptor.Builder()
+            .name("delete-snapshots-option")
+            .displayName("Delete Snapshots Option")
+            .description("Specifies the snapshot deletion options to be used when deleting a blob.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .allowableValues(DELETE_SNAPSHOTS_NONE, DELETE_SNAPSHOTS_ALSO, DELETE_SNAPSHOTS_ONLY)
+            .defaultValue(DELETE_SNAPSHOTS_NONE.getValue())
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            BLOB_NAME,
+            DELETE_SNAPSHOTS_OPTION
+    ));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String containerName = context.getProperty(AzureStorageUtils.CONTAINER).evaluateAttributeExpressions(flowFile).getValue();
+        String blobName = context.getProperty(BLOB_NAME).evaluateAttributeExpressions(flowFile).getValue();
+        String deleteSnapshotsOption = context.getProperty(DELETE_SNAPSHOTS_OPTION).getValue();
+
+        long startNanos = System.nanoTime();
+        try {
+            BlobServiceClient storageClient = getStorageClient();
+            BlobContainerClient containerClient = storageClient.getBlobContainerClient(containerName);
+            BlobClient blobClient = containerClient.getBlobClient(blobName);
+
+            String provenanceMesage;
+            if (blobClient.exists()) {
+                DeleteSnapshotsOptionType deleteSnapshotsOptionType = getDeleteSnapshotsOptionType(deleteSnapshotsOption);
+                blobClient.deleteWithResponse(deleteSnapshotsOptionType, null, null, null);
+                provenanceMesage = getProvenanceMessage(deleteSnapshotsOptionType);
+            } else {
+                provenanceMesage = "Blob does not exist, nothing to delete";
+            }
+
+            session.transfer(flowFile, REL_SUCCESS);
+
+            long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+            session.getProvenanceReporter().invokeRemoteProcess(flowFile, blobClient.getBlobUrl(), String.format("%s (%d ms)", provenanceMesage, transferMillis));
+        } catch ( Exception e) {

Review comment:
       Fixed, thanks.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.azure;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_TIMESTAMP;
+
+public abstract class AbstractAzureBlobProcessor_v12 extends AbstractProcessor {
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("blob-name")
+            .displayName("Blob Name")
+            .description("The full name of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

Review comment:
       Fixed, thanks.




-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageCredentialsControllerService_v12.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides credentials details for Azure Blob processors
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "blob", "credentials"})
+@CapabilityDescription("Provides credentials for Azure Blob processors using Azure Blob Storage client library v12.")
+public class AzureStorageCredentialsControllerService_v12 extends AbstractControllerService implements AzureStorageCredentialsService_v12 {
+
+    public static final String DEFAULT_ENDPOINT_SUFFIX = "blob.core.windows.net";
+
+    public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AzureStorageUtils.ACCOUNT_NAME)
+            .description(AzureStorageUtils.ACCOUNT_NAME_BASE_DESCRIPTION)
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor 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(DEFAULT_ENDPOINT_SUFFIX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor CREDENTIALS_TYPE = new PropertyDescriptor.Builder()
+            .name("credentials-type")
+            .displayName("Credentials Type")
+            .description("Credentials type to be used for authenticating to Azure")
+            .required(true)
+            .allowableValues(AzureStorageCredentialsType.getAllowableValues())
+            .defaultValue(AzureStorageCredentialsType.ACCOUNT_KEY.name())

Review comment:
       I would vote for SAS Token and have set it. Managed Identity is a more specific case when NiFi is running on an Azure VM.




-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureBlobStorage_v12IT.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.processors.azure.storage;
+
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobType;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.processors.azure.AbstractAzureBlobProcessor_v12;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.processors.azure.storage.utils.BlobAttributes;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsControllerService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.After;
+import org.junit.Before;
+
+import java.io.ByteArrayInputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public abstract class AbstractAzureBlobStorage_v12IT extends AbstractAzureStorageIT {
+
+    protected static final String BLOB_NAME = "blob1";
+    protected static final byte[] BLOB_DATA = "0123456789".getBytes();
+
+    protected static final String EL_CONTAINER_NAME = "az.containername";
+    protected static final String EL_BLOB_NAME = "az.blobname";
+
+    protected static final byte[] EMPTY_CONTENT = new byte[0];
+
+    private static final String TEST_CONTAINER_NAME_PREFIX = "nifi-test-container";
+
+    private BlobServiceClient storageClient;
+    private BlobContainerClient containerClient;
+    private String containerName;
+
+    @Override
+    protected void setUpCredentials() throws Exception {
+        String serviceId = "credentials-service";
+        AzureStorageCredentialsService_v12 service = new AzureStorageCredentialsControllerService_v12();
+        runner.addControllerService(serviceId, service);
+        runner.setProperty(service, AzureStorageCredentialsControllerService_v12.ACCOUNT_NAME, getAccountName());
+        runner.setProperty(service, AzureStorageCredentialsControllerService_v12.ACCOUNT_KEY, getAccountKey());
+        runner.enableControllerService(service);
+
+        runner.setProperty(AbstractAzureBlobProcessor_v12.STORAGE_CREDENTIALS_SERVICE, serviceId);
+    }
+
+    @Before
+    public void setUpAzureBlobStorage_v12IT() {
+        containerName = generateContainerName();
+
+        runner.setProperty(AzureStorageUtils.CONTAINER, containerName);
+
+        storageClient = createStorageClient();
+        containerClient = storageClient.createBlobContainer(containerName);
+    }
+
+    @After
+    public void tearDownAzureBlobStorage_v12IT() {
+        containerClient.delete();
+    }
+
+    protected String generateContainerName() {
+        return String.format("%s-%s", TEST_CONTAINER_NAME_PREFIX, UUID.randomUUID());
+    }
+
+    protected BlobServiceClient getStorageClient() {
+        return storageClient;
+    }
+
+    protected BlobContainerClient getContainerClient() {
+        return containerClient;
+    }
+
+    protected String getContainerName() {
+        return containerName;
+    }
+
+    private BlobServiceClient createStorageClient() {
+        return new BlobServiceClientBuilder()
+                .endpoint("https://" + getAccountName() + ".blob.core.windows.net")

Review comment:
       "endpointSuffix" property can now be configured in azure-credentials.PROPERTIES for the IT tests. If not specified, the default endpoint will be used.




-- 
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] asfgit closed pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5486:
URL: https://github.com/apache/nifi/pull/5486


   


-- 
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 #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.azure;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_TIMESTAMP;
+
+public abstract class AbstractAzureBlobProcessor_v12 extends AbstractProcessor {
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("blob-name")
+            .displayName("Blob Name")
+            .description("The full name of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("All successfully processed FlowFiles are routed to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Unsuccessful operations will be transferred to the failure relationship.")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    private BlobServiceClient storageClient;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        storageClient = createStorageClient(context);
+    }
+
+    @OnStopped
+    public void onStopped() {
+        storageClient = null;
+    }
+
+    protected BlobServiceClient getStorageClient() {
+        return storageClient;
+    }
+
+    public static BlobServiceClient createStorageClient(PropertyContext context) {

Review comment:
       Generally speaking I think it'd be good to not be returning any of the types defined in the SDK. This is part of the issue requiring the _v12 variant and I think it's safer to hide this from the user or making it protected.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/DeleteAzureBlobStorage_v12.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.processors.azure.storage;
+
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.DeleteSnapshotsOptionType;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AbstractAzureBlobProcessor_v12;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@Tags({ "azure", "microsoft", "cloud", "storage", "blob" })
+@SeeAlso({ ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, PutAzureBlobStorage_v12.class})
+@CapabilityDescription("Deletes the specified blob from Azure Blob Storage. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+public class DeleteAzureBlobStorage_v12 extends AbstractAzureBlobProcessor_v12 {
+
+    public static final AllowableValue DELETE_SNAPSHOTS_NONE = new AllowableValue("NONE", "None", "Delete the blob only.");
+
+    public static final AllowableValue DELETE_SNAPSHOTS_ALSO = new AllowableValue(DeleteSnapshotsOptionType.INCLUDE.name(), "Include Snapshots", "Delete the blob and its snapshots.");
+
+    public static final AllowableValue DELETE_SNAPSHOTS_ONLY = new AllowableValue(DeleteSnapshotsOptionType.ONLY.name(), "Delete Snapshots Only", "Delete only the blob's snapshots.");
+
+    public static final PropertyDescriptor DELETE_SNAPSHOTS_OPTION = new PropertyDescriptor.Builder()
+            .name("delete-snapshots-option")
+            .displayName("Delete Snapshots Option")
+            .description("Specifies the snapshot deletion options to be used when deleting a blob.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .allowableValues(DELETE_SNAPSHOTS_NONE, DELETE_SNAPSHOTS_ALSO, DELETE_SNAPSHOTS_ONLY)
+            .defaultValue(DELETE_SNAPSHOTS_NONE.getValue())
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            BLOB_NAME,
+            DELETE_SNAPSHOTS_OPTION
+    ));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String containerName = context.getProperty(AzureStorageUtils.CONTAINER).evaluateAttributeExpressions(flowFile).getValue();
+        String blobName = context.getProperty(BLOB_NAME).evaluateAttributeExpressions(flowFile).getValue();
+        String deleteSnapshotsOption = context.getProperty(DELETE_SNAPSHOTS_OPTION).getValue();
+
+        long startNanos = System.nanoTime();
+        try {
+            BlobServiceClient storageClient = getStorageClient();
+            BlobContainerClient containerClient = storageClient.getBlobContainerClient(containerName);
+            BlobClient blobClient = containerClient.getBlobClient(blobName);
+
+            String provenanceMesage;
+            if (blobClient.exists()) {
+                DeleteSnapshotsOptionType deleteSnapshotsOptionType = getDeleteSnapshotsOptionType(deleteSnapshotsOption);
+                blobClient.deleteWithResponse(deleteSnapshotsOptionType, null, null, null);
+                provenanceMesage = getProvenanceMessage(deleteSnapshotsOptionType);
+            } else {
+                provenanceMesage = "Blob does not exist, nothing to delete";
+            }
+
+            session.transfer(flowFile, REL_SUCCESS);
+
+            long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+            session.getProvenanceReporter().invokeRemoteProcess(flowFile, blobClient.getBlobUrl(), String.format("%s (%d ms)", provenanceMesage, transferMillis));
+        } catch ( Exception e) {

Review comment:
       ```suggestion
           } catch (Exception e) {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.azure;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_TIMESTAMP;
+
+public abstract class AbstractAzureBlobProcessor_v12 extends AbstractProcessor {
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("blob-name")
+            .displayName("Blob Name")
+            .description("The full name of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

Review comment:
       ```suggestion
               .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageCredentialsControllerService_v12.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides credentials details for Azure Blob processors
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "blob", "credentials"})
+@CapabilityDescription("Provides credentials for Azure Blob processors using Azure Blob Storage client library v12.")
+public class AzureStorageCredentialsControllerService_v12 extends AbstractControllerService implements AzureStorageCredentialsService_v12 {
+
+    public static final String DEFAULT_ENDPOINT_SUFFIX = "blob.core.windows.net";

Review comment:
       This isn't specific to your improvement but default endpoints are all over the Azure nar package and I think it'd be useful to pull these out into a set of constants or make use of [com.azure.core.management.AzureEnvironment](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-management/src/main/java/com/azure/core/management/AzureEnvironment.java) or some combination so we're not dependent on the stability of the SDK.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageCredentialsControllerService_v12.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides credentials details for Azure Blob processors
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "blob", "credentials"})
+@CapabilityDescription("Provides credentials for Azure Blob processors using Azure Blob Storage client library v12.")
+public class AzureStorageCredentialsControllerService_v12 extends AbstractControllerService implements AzureStorageCredentialsService_v12 {
+
+    public static final String DEFAULT_ENDPOINT_SUFFIX = "blob.core.windows.net";
+
+    public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AzureStorageUtils.ACCOUNT_NAME)
+            .description(AzureStorageUtils.ACCOUNT_NAME_BASE_DESCRIPTION)
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor 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(DEFAULT_ENDPOINT_SUFFIX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor CREDENTIALS_TYPE = new PropertyDescriptor.Builder()
+            .name("credentials-type")
+            .displayName("Credentials Type")
+            .description("Credentials type to be used for authenticating to Azure")
+            .required(true)
+            .allowableValues(AzureStorageCredentialsType.getAllowableValues())
+            .defaultValue(AzureStorageCredentialsType.ACCOUNT_KEY.name())

Review comment:
       I think there's an opportunity here to push people toward better security practices, making the default managed identity or SAS tokens. What do you think?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureBlobStorage_v12IT.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.processors.azure.storage;
+
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobType;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.processors.azure.AbstractAzureBlobProcessor_v12;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+import org.apache.nifi.processors.azure.storage.utils.BlobAttributes;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsControllerService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.After;
+import org.junit.Before;
+
+import java.io.ByteArrayInputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public abstract class AbstractAzureBlobStorage_v12IT extends AbstractAzureStorageIT {
+
+    protected static final String BLOB_NAME = "blob1";
+    protected static final byte[] BLOB_DATA = "0123456789".getBytes();
+
+    protected static final String EL_CONTAINER_NAME = "az.containername";
+    protected static final String EL_BLOB_NAME = "az.blobname";
+
+    protected static final byte[] EMPTY_CONTENT = new byte[0];
+
+    private static final String TEST_CONTAINER_NAME_PREFIX = "nifi-test-container";
+
+    private BlobServiceClient storageClient;
+    private BlobContainerClient containerClient;
+    private String containerName;
+
+    @Override
+    protected void setUpCredentials() throws Exception {
+        String serviceId = "credentials-service";
+        AzureStorageCredentialsService_v12 service = new AzureStorageCredentialsControllerService_v12();
+        runner.addControllerService(serviceId, service);
+        runner.setProperty(service, AzureStorageCredentialsControllerService_v12.ACCOUNT_NAME, getAccountName());
+        runner.setProperty(service, AzureStorageCredentialsControllerService_v12.ACCOUNT_KEY, getAccountKey());
+        runner.enableControllerService(service);
+
+        runner.setProperty(AbstractAzureBlobProcessor_v12.STORAGE_CREDENTIALS_SERVICE, serviceId);
+    }
+
+    @Before
+    public void setUpAzureBlobStorage_v12IT() {
+        containerName = generateContainerName();
+
+        runner.setProperty(AzureStorageUtils.CONTAINER, containerName);
+
+        storageClient = createStorageClient();
+        containerClient = storageClient.createBlobContainer(containerName);
+    }
+
+    @After
+    public void tearDownAzureBlobStorage_v12IT() {
+        containerClient.delete();
+    }
+
+    protected String generateContainerName() {
+        return String.format("%s-%s", TEST_CONTAINER_NAME_PREFIX, UUID.randomUUID());
+    }
+
+    protected BlobServiceClient getStorageClient() {
+        return storageClient;
+    }
+
+    protected BlobContainerClient getContainerClient() {
+        return containerClient;
+    }
+
+    protected String getContainerName() {
+        return containerName;
+    }
+
+    private BlobServiceClient createStorageClient() {
+        return new BlobServiceClientBuilder()
+                .endpoint("https://" + getAccountName() + ".blob.core.windows.net")

Review comment:
       This will prohibit using the ITs on national clouds (e.g., USGov, Germany, China). I think we should expose this as a property.




-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.azure;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_TIMESTAMP;
+
+public abstract class AbstractAzureBlobProcessor_v12 extends AbstractProcessor {
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("blob-name")
+            .displayName("Blob Name")
+            .description("The full name of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("All successfully processed FlowFiles are routed to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Unsuccessful operations will be transferred to the failure relationship.")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    private BlobServiceClient storageClient;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        storageClient = createStorageClient(context);
+    }
+
+    @OnStopped
+    public void onStopped() {
+        storageClient = null;
+    }
+
+    protected BlobServiceClient getStorageClient() {
+        return storageClient;
+    }
+
+    public static BlobServiceClient createStorageClient(PropertyContext context) {

Review comment:
       I'm afraid we could not hide it totally. Even if it was `protected`, someone could use this abstract class as a parent class for their custom processors. Unfortunately, we will not be able to change it due to backward compatibility.
   
   Furthermore, it is `public` and `static` because of the Listen processor. Put, Fetch and Delete processors are subclasses of this abstract class (so `protected` would be fine) but Listen should come from `AbstractListProcessor`. For this reason, it accesses this method in static way. I know it is weird but the existing blob/adls List processors use the same and I have no better idea. I tried to subclass the Listen processor from the blob abstract class and define an `AbstractListProcessor` sublcass in the Listen processor (as an inner class) and use it as a delegate but it is not straightforward because all the processor lifecycle method callbacks should be delegated as well. So I gave up this approach.
   Any idea would be appreciated!




-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.azure;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_TIMESTAMP;
+
+public abstract class AbstractAzureBlobProcessor_v12 extends AbstractProcessor {
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("blob-name")
+            .displayName("Blob Name")
+            .description("The full name of the blob")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("All successfully processed FlowFiles are routed to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Unsuccessful operations will be transferred to the failure relationship.")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    private BlobServiceClient storageClient;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        storageClient = createStorageClient(context);
+    }
+
+    @OnStopped
+    public void onStopped() {
+        storageClient = null;
+    }
+
+    protected BlobServiceClient getStorageClient() {
+        return storageClient;
+    }
+
+    public static BlobServiceClient createStorageClient(PropertyContext context) {

Review comment:
       I'm afraid we cannot not hide it totally. Even if it was `protected`, someone could use this abstract class as a parent class for their custom processors. Unfortunately, we will not be able to change it due to backward compatibility.
   
   Furthermore, it is `public` and `static` because of the Listen processor. Put, Fetch and Delete processors are subclasses of this abstract class (so `protected` would be fine) but Listen should come from `AbstractListProcessor`. For this reason, it accesses this method in static way. I know it is weird but the existing blob/adls List processors use the same and I have no better idea. I tried to subclass the List processor from the blob abstract class and define an `AbstractListProcessor` sublcass in the List processor (as an inner class) and use it as a delegate but it is not straightforward because all the processor lifecycle method callbacks should be delegated as well. So I gave up this approach.
   Any idea would be appreciated!




-- 
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] turcsanyip commented on a change in pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/services/azure/storage/AzureStorageCredentialsControllerService_v12.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.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.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Provides credentials details for Azure Blob processors
+ *
+ * @see AbstractControllerService
+ */
+@Tags({"azure", "microsoft", "cloud", "storage", "blob", "credentials"})
+@CapabilityDescription("Provides credentials for Azure Blob processors using Azure Blob Storage client library v12.")
+public class AzureStorageCredentialsControllerService_v12 extends AbstractControllerService implements AzureStorageCredentialsService_v12 {
+
+    public static final String DEFAULT_ENDPOINT_SUFFIX = "blob.core.windows.net";

Review comment:
       Added AzureServiceEndpoints to provide endpoint suffix constants and methods to get endpoint urls.




-- 
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 pull request #5486: NIFI-9338: Add Azure Blob processors using Azure Blob Storage client …

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


   @turcsanyip Thanks for the updates. I tested several scenarios and it looks good.


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