You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "nandorsoma (via GitHub)" <gi...@apache.org> on 2023/03/09 10:32:36 UTC

[GitHub] [nifi] nandorsoma commented on a diff in pull request #6941: NIFI-9972 Azure Blob Storage blob copying

nandorsoma commented on code in PR #6941:
URL: https://github.com/apache/nifi/pull/6941#discussion_r1129517795


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()

Review Comment:
   I'd call this property DESTINATION_STORAGE_CRENDENTIALS_SERVICE.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()

Review Comment:
   Similarly this one as DESTINATION_CONTAINER.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,

Review Comment:
   I'd group the properties a bit differently. I think it is easier to understand if you move the source properties to the top of the list, then the "destination" properties could follow them that come from the abstract class. Speaking of which I'd also "override" them in this class and I'd add a modified description and a displayName startig with `Destination`. You can see an example here: https://github.com/apache/nifi/blob/5088934b79e39e4180464186020dc62a0240df37/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/MoveAzureDataLakeStorage.java#L114



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureBlobProcessor_v12.java:
##########
@@ -113,25 +115,24 @@ protected BlobServiceClient getStorageClient() {
         return storageClient;
     }
 
-    public static BlobServiceClient createStorageClient(PropertyContext context) {
-        final AzureStorageCredentialsService_v12 credentialsService = context.getProperty(STORAGE_CREDENTIALS_SERVICE).asControllerService(AzureStorageCredentialsService_v12.class);
+    public static BlobServiceClient createStorageClient(HttpClient nettyClient, AzureStorageCredentialsService_v12 credentialsService) {

Review Comment:
   While generally, code decoupling is good, I wonder why it was needed in this situation. Is there something I didn't notice? Because, in my understanding, the behavior remained the same.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            CREATE_CONTAINER,
+            CONFLICT_RESOLUTION,
+            BLOB_NAME,
+            COPY_FROM_STORAGE_CREDENTIALS_SERVICE,
+            COPY_FROM_CONTAINER,
+            COPY_FROM_BLOB_NAME,
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private BlobServiceClient fromStorageClient;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        super.onScheduled(context);
+        final AzureStorageCredentialsService_v12 credentialsService = getCopyFromCredentialsService(context);
+        if (credentialsService != null) {
+            final HttpClient nettyClient = createHttpClient(context);
+            fromStorageClient = createFromStorageClient(nettyClient, credentialsService);
+        }
+    }
+
+    @OnStopped
+    public void onStopped() {
+        super.onStopped();
+        fromStorageClient = null;
+    }
+
+    private BlobServiceClient createFromStorageClient(final HttpClient nettyClient, final AzureStorageCredentialsService_v12 credentialsService) {
+        return createStorageClient(nettyClient, credentialsService);
+    }
+
+    private BlobServiceClient getFromStorageClient() {
+        return fromStorageClient;
+    }
+
+    private static AzureStorageCredentialsService_v12 getCopyFromCredentialsService(ProcessContext context) {

Review Comment:
   I think this method is not needed and you can use directly `context.getProperty(...)` instead.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            CREATE_CONTAINER,
+            CONFLICT_RESOLUTION,
+            BLOB_NAME,
+            COPY_FROM_STORAGE_CREDENTIALS_SERVICE,
+            COPY_FROM_CONTAINER,
+            COPY_FROM_BLOB_NAME,
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private BlobServiceClient fromStorageClient;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        super.onScheduled(context);
+        final AzureStorageCredentialsService_v12 credentialsService = getCopyFromCredentialsService(context);
+        if (credentialsService != null) {
+            final HttpClient nettyClient = createHttpClient(context);
+            fromStorageClient = createFromStorageClient(nettyClient, credentialsService);
+        }
+    }
+
+    @OnStopped
+    public void onStopped() {
+        super.onStopped();
+        fromStorageClient = null;
+    }
+
+    private BlobServiceClient createFromStorageClient(final HttpClient nettyClient, final AzureStorageCredentialsService_v12 credentialsService) {

Review Comment:
   I think this method is not needed and you can call `createStorageClient()` directly.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            CREATE_CONTAINER,
+            CONFLICT_RESOLUTION,
+            BLOB_NAME,
+            COPY_FROM_STORAGE_CREDENTIALS_SERVICE,
+            COPY_FROM_CONTAINER,
+            COPY_FROM_BLOB_NAME,
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private BlobServiceClient fromStorageClient;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        super.onScheduled(context);
+        final AzureStorageCredentialsService_v12 credentialsService = getCopyFromCredentialsService(context);
+        if (credentialsService != null) {
+            final HttpClient nettyClient = createHttpClient(context);
+            fromStorageClient = createFromStorageClient(nettyClient, credentialsService);
+        }
+    }
+
+    @OnStopped
+    public void onStopped() {
+        super.onStopped();
+        fromStorageClient = null;
+    }
+
+    private BlobServiceClient createFromStorageClient(final HttpClient nettyClient, final AzureStorageCredentialsService_v12 credentialsService) {
+        return createStorageClient(nettyClient, credentialsService);
+    }
+
+    private BlobServiceClient getFromStorageClient() {
+        return fromStorageClient;
+    }
+
+    private static AzureStorageCredentialsService_v12 getCopyFromCredentialsService(ProcessContext context) {
+        return context.getProperty(COPY_FROM_STORAGE_CREDENTIALS_SERVICE).asControllerService(AzureStorageCredentialsService_v12.class);
+    }
+
+    protected Map<String, String> putBlob(ProcessSession session,

Review Comment:
   Override is missing, but I think this method should be renamed because it has nothing to do with putting a blob. It is more like a preconfiguration for the copy method.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            CREATE_CONTAINER,
+            CONFLICT_RESOLUTION,
+            BLOB_NAME,
+            COPY_FROM_STORAGE_CREDENTIALS_SERVICE,
+            COPY_FROM_CONTAINER,
+            COPY_FROM_BLOB_NAME,
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private BlobServiceClient fromStorageClient;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        super.onScheduled(context);
+        final AzureStorageCredentialsService_v12 credentialsService = getCopyFromCredentialsService(context);
+        if (credentialsService != null) {
+            final HttpClient nettyClient = createHttpClient(context);
+            fromStorageClient = createFromStorageClient(nettyClient, credentialsService);
+        }
+    }
+
+    @OnStopped
+    public void onStopped() {
+        super.onStopped();
+        fromStorageClient = null;
+    }
+
+    private BlobServiceClient createFromStorageClient(final HttpClient nettyClient, final AzureStorageCredentialsService_v12 credentialsService) {
+        return createStorageClient(nettyClient, credentialsService);
+    }
+
+    private BlobServiceClient getFromStorageClient() {
+        return fromStorageClient;
+    }
+
+    private static AzureStorageCredentialsService_v12 getCopyFromCredentialsService(ProcessContext context) {
+        return context.getProperty(COPY_FROM_STORAGE_CREDENTIALS_SERVICE).asControllerService(AzureStorageCredentialsService_v12.class);
+    }
+
+    protected Map<String, String> putBlob(ProcessSession session,
+                                          ProcessContext context,
+                                          FlowFile flowFile,
+                                          BlobClient blobClient,
+                                          BlobRequestConditions destinationRequestConditions) throws IOException {
+        final BlobServiceClient fromStorageClient = getFromStorageClient();
+
+        final AzureStorageCredentialsService_v12 fromCredentialsService = getCopyFromCredentialsService(context);
+        String fromContainerName =
+                Optional.ofNullable(context.getProperty(COPY_FROM_CONTAINER).evaluateAttributeExpressions(flowFile).getValue())
+                        .orElse(blobClient.getContainerName());
+        String fromBlobName =
+                Optional.ofNullable(context.getProperty(COPY_FROM_BLOB_NAME).evaluateAttributeExpressions(flowFile).getValue())
+                        .orElse(blobClient.getBlobName());
+
+        BlobContainerClient fromContainerClient = fromStorageClient.getBlobContainerClient(fromContainerName);
+        BlobClient fromBlobClient = fromContainerClient.getBlobClient(fromBlobName);
+
+        AzureStorageCredentialsDetails_v12 credentialsDetails = fromCredentialsService.getCredentialsDetails();
+        String sourceUrl = fromBlobClient.getBlobUrl();
+        final BlobProperties fromBlobProperties = fromBlobClient.getProperties();
+        final long blobSize = fromBlobProperties.getBlobSize();
+
+        final BlobRequestConditions sourceRequestConditions = new BlobRequestConditions();
+        sourceRequestConditions.setIfMatch(fromBlobProperties.getETag());
+
+        HttpAuthorization httpAuthorization;
+        final String sasToken = (credentialsDetails.getCredentialsType() == AzureStorageCredentialsType.ACCOUNT_KEY)
+                ? generateSas(fromContainerClient, GENERATE_SAS_EXPIRY_HOURS)
+                : credentialsDetails.getSasToken();
+        if (sasToken != null) {
+            sourceUrl += "?" + sasToken;
+            httpAuthorization = null;
+        } else {
+            httpAuthorization = getHttpAuthorization(credentialsDetails);
+        }
+
+        copy(blobClient, httpAuthorization, sourceUrl, blobSize, sourceRequestConditions, destinationRequestConditions);
+        return null;
+    }
+
+    private static String generateSas(final BlobContainerClient fromContainerClient, final int expiryHours) {
+        BlobContainerSasPermission permissions = new BlobContainerSasPermission().setCreatePermission(true).setWritePermission(true).setAddPermission(true).setReadPermission(true);
+        OffsetDateTime utc = OffsetDateTime.now(ZoneOffset.UTC);
+        OffsetDateTime expiryTime = utc.plus(expiryHours, ChronoUnit.HOURS);
+        BlobServiceSasSignatureValues signatureValues = new BlobServiceSasSignatureValues(expiryTime, permissions);
+        return fromContainerClient.generateSas(signatureValues);
+    }
+
+    private void copy(final BlobClient blobClient,
+                      final HttpAuthorization httpAuthorization,
+                      final String sourceUrl,
+                      long blobSize,
+                      final BlobRequestConditions sourceRequestConditions,
+                      final BlobRequestConditions destinationRequestConditions) {
+        BlockBlobClient blockBlobClient = blobClient.getBlockBlobClient();
+
+        // If the blob size is below the limit, we use the one-shot upload endpoint.
+        if (blobSize < MAX_UPLOAD_BLOB_BYTES_LONG) {
+            final BlobUploadFromUrlOptions options = new BlobUploadFromUrlOptions(sourceUrl);
+            if (httpAuthorization != null) {
+                options.setSourceAuthorization(httpAuthorization);
+            }
+            options.setSourceRequestConditions(sourceRequestConditions);
+            options.setDestinationRequestConditions(destinationRequestConditions);
+            blockBlobClient.uploadFromUrlWithResponse(options, null, Context.NONE);
+            return;
+        }
+
+        final DecimalFormat df = new DecimalFormat("0000000");
+        long offset = 0;
+        int blockId = 1;
+        final List<String> blockIds = new ArrayList<>();
+
+        // Upload each block sequentially until we've uploaded the entire blob.
+        while (true) {
+            long count = Math.min(blobSize - offset, MAX_STAGE_BLOCK_BYTES_LONG);
+            if (count == 0) break;
+
+            final String base64BlockId = Base64.getEncoder().encodeToString(df.format(blockId).getBytes());
+            BlockBlobStageBlockFromUrlOptions blockBlobStageBlockFromUrlOptions = new BlockBlobStageBlockFromUrlOptions(base64BlockId, sourceUrl);
+            blockBlobStageBlockFromUrlOptions.setSourceRange(new BlobRange(offset, count));
+
+            if (httpAuthorization != null) {
+                blockBlobStageBlockFromUrlOptions.setSourceAuthorization(httpAuthorization);
+            }
+            blockBlobStageBlockFromUrlOptions.setSourceRequestConditions(sourceRequestConditions);
+            final int statusCode = blockBlobClient.stageBlockFromUrlWithResponse(blockBlobStageBlockFromUrlOptions, null, Context.NONE).getStatusCode();
+            if (statusCode != 201) {
+                throw new ProcessException(String.format("Failed staging one or more blocks (status: %d)", statusCode));
+            }
+            blockIds.add(base64BlockId);
+            offset += count;
+            blockId++;
+        }
+
+        final BlockBlobCommitBlockListOptions options = new BlockBlobCommitBlockListOptions(blockIds);
+        options.setRequestConditions(destinationRequestConditions);
+        final Response<BlockBlobItem> response = blockBlobClient.commitBlockListWithResponse(options, null, Context.NONE);
+        final int statusCode = response.getStatusCode();
+        if (statusCode != 201) {
+            throw new ProcessException(String.format("Failed committing block list (status: %d)", statusCode));
+        }
+    }
+
+    private static HttpAuthorization getHttpAuthorization(final AzureStorageCredentialsDetails_v12 credentialsDetails) {

Review Comment:
   I feel like this authentication-related code shouldn't belong here because we have something similar in the abstract class. After a quick glance, I don't know a good solution, but it is worth investigating to merge those code parts somehow.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/PutAzureBlobStorage_v12.java:
##########
@@ -199,12 +197,26 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
         }
     }
 
-    private static void applyUploadResultAttributes(final Map<String, String> attributes, final BlockBlobItem blob, final BlobType blobType, final long length) {
-        attributes.put(ATTR_NAME_BLOBTYPE, blobType.toString());
+    protected Map<String, String> putBlob(ProcessSession session,

Review Comment:
   I think if you don't extend this class, this refactor won't be needed at all.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {

Review Comment:
   What is the benefit of extending `PutAzureBlobStorage_v12` instead of `AbstractAzureBlobProcessor_v12`? If I understand correctly, you are overriding anyway the only method that is added by that class.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()

Review Comment:
   Similarly this one as DESTINATION_BLOB_NAME.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITCopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.models.BlobErrorCode;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processors.azure.storage.CopyAzureBlobStorage_v12.COPY_FROM_STORAGE_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_NAME_ERROR_CODE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ITCopyAzureBlobStorage_v12 extends AbstractAzureBlobStorage_v12IT {

Review Comment:
   I know it is not realistic to test all authentication types, but I feel uneasy that you mentioned that you have tested only with the `Account key`, while I see that a significant part of the code is related to `SAS token`. Can I ask you to create a test with that authentication method?



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/CopyAzureBlobStorage_v12.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.core.credential.AccessToken;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.credential.TokenRequestContext;
+import com.azure.core.http.HttpAuthorization;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.rest.Response;
+import com.azure.core.util.Context;
+import com.azure.identity.ClientSecretCredential;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredential;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobProperties;
+import com.azure.storage.blob.models.BlobRange;
+import com.azure.storage.blob.models.BlobRequestConditions;
+import com.azure.storage.blob.models.BlockBlobItem;
+import com.azure.storage.blob.options.BlobUploadFromUrlOptions;
+import com.azure.storage.blob.options.BlockBlobCommitBlockListOptions;
+import com.azure.storage.blob.options.BlockBlobStageBlockFromUrlOptions;
+import com.azure.storage.blob.sas.BlobContainerSasPermission;
+import com.azure.storage.blob.sas.BlobServiceSasSignatureValues;
+import com.azure.storage.blob.specialized.BlockBlobClient;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.storage.utils.AzureStorageUtils;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsDetails_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsService_v12;
+import org.apache.nifi.services.azure.storage.AzureStorageCredentialsType;
+import reactor.core.publisher.Mono;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_STAGE_BLOCK_BYTES_LONG;
+import static com.azure.storage.blob.specialized.BlockBlobClient.MAX_UPLOAD_BLOB_BYTES_LONG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBNAME;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_BLOBTYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_CONTAINER;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ERROR_CODE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_ETAG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_IGNORED;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LANG;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_LENGTH;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_MIME_TYPE;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_PRIMARY_URI;
+import static org.apache.nifi.processors.azure.storage.utils.BlobAttributes.ATTR_DESCRIPTION_TIMESTAMP;
+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_ERROR_CODE;
+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_IGNORED;
+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;
+
+@Tags({"azure", "microsoft", "cloud", "storage", "blob"})
+@SeeAlso({ListAzureBlobStorage_v12.class, FetchAzureBlobStorage_v12.class, DeleteAzureBlobStorage_v12.class})
+@CapabilityDescription("Copies a blob in Azure Blob Storage from one account/container to another. The processor uses Azure Blob Storage client library v12.")
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@WritesAttributes({@WritesAttribute(attribute = ATTR_NAME_CONTAINER, description = ATTR_DESCRIPTION_CONTAINER),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBNAME, description = ATTR_DESCRIPTION_BLOBNAME),
+        @WritesAttribute(attribute = ATTR_NAME_PRIMARY_URI, description = ATTR_DESCRIPTION_PRIMARY_URI),
+        @WritesAttribute(attribute = ATTR_NAME_ETAG, description = ATTR_DESCRIPTION_ETAG),
+        @WritesAttribute(attribute = ATTR_NAME_BLOBTYPE, description = ATTR_DESCRIPTION_BLOBTYPE),
+        @WritesAttribute(attribute = ATTR_NAME_MIME_TYPE, description = ATTR_DESCRIPTION_MIME_TYPE),
+        @WritesAttribute(attribute = ATTR_NAME_LANG, description = ATTR_DESCRIPTION_LANG),
+        @WritesAttribute(attribute = ATTR_NAME_TIMESTAMP, description = ATTR_DESCRIPTION_TIMESTAMP),
+        @WritesAttribute(attribute = ATTR_NAME_LENGTH, description = ATTR_DESCRIPTION_LENGTH),
+        @WritesAttribute(attribute = ATTR_NAME_ERROR_CODE, description = ATTR_DESCRIPTION_ERROR_CODE),
+        @WritesAttribute(attribute = ATTR_NAME_IGNORED, description = ATTR_DESCRIPTION_IGNORED)})
+public class CopyAzureBlobStorage_v12 extends PutAzureBlobStorage_v12 {
+    private final static int GENERATE_SAS_EXPIRY_HOURS = 24;
+
+    public static final PropertyDescriptor COPY_FROM_STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("copy-from-storage-credentials-service")
+            .displayName("Copy From Storage Credentials")
+            .description("Controller Service used to obtain Azure Blob Storage Credentials to read blob data ")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_CONTAINER = new PropertyDescriptor.Builder()
+            .name("copy-from-container-name")
+            .displayName("Copy From Container Name")
+            .description("If specified, copy from the specific container instead of using FlowFile content.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    public static final PropertyDescriptor COPY_FROM_BLOB_NAME = new PropertyDescriptor.Builder()
+            .name("copy-from-blob-name")
+            .displayName("Copy From Blob Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .description("If specified, copy from the specified blob name instead of using FlowFile content.")
+            .dependsOn(COPY_FROM_STORAGE_CREDENTIALS_SERVICE)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            STORAGE_CREDENTIALS_SERVICE,
+            AzureStorageUtils.CONTAINER,
+            CREATE_CONTAINER,
+            CONFLICT_RESOLUTION,
+            BLOB_NAME,
+            COPY_FROM_STORAGE_CREDENTIALS_SERVICE,
+            COPY_FROM_CONTAINER,
+            COPY_FROM_BLOB_NAME,
+            AzureStorageUtils.PROXY_CONFIGURATION_SERVICE
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private BlobServiceClient fromStorageClient;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        super.onScheduled(context);
+        final AzureStorageCredentialsService_v12 credentialsService = getCopyFromCredentialsService(context);
+        if (credentialsService != null) {
+            final HttpClient nettyClient = createHttpClient(context);
+            fromStorageClient = createFromStorageClient(nettyClient, credentialsService);
+        }
+    }
+
+    @OnStopped
+    public void onStopped() {
+        super.onStopped();
+        fromStorageClient = null;
+    }
+
+    private BlobServiceClient createFromStorageClient(final HttpClient nettyClient, final AzureStorageCredentialsService_v12 credentialsService) {
+        return createStorageClient(nettyClient, credentialsService);
+    }
+
+    private BlobServiceClient getFromStorageClient() {

Review Comment:
   I think this method is not needed as well.



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