You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "exceptionfactory (via GitHub)" <gi...@apache.org> on 2023/05/25 16:22:42 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7269: NIFI-11549: implemented AzureQueueStorage_v12 processors

exceptionfactory commented on code in PR #7269:
URL: https://github.com/apache/nifi/pull/7269#discussion_r1205729048


##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/queue/TestGetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.queue;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestGetAzureQueueStorage_v12 extends AbstractTestAzureQueueStorage_v12 {
+    @BeforeEach
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(GetAzureQueueStorage_v12.class);
+        setupStorageCredentialsService();
+        runner.enableControllerService(credentialsService);
+        runner.setProperty(GetAzureQueueStorage_v12.STORAGE_CREDENTIALS_SERVICE, CREDENTIALS_SERVICE_IDENTIFIER);
+    }
+
+    @Test
+    public void testValidVisibilityTimeout() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "10 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutZeroSecs() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "0 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("0 secs"));
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutMoreThanSevenDays() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "8 days");

Review Comment:
   The repetitive property setting should be moved to a shared method



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/PutAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueStorageException;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+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.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.ByteArrayOutputStream;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({GetAzureQueueStorage_v12.class})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({ "azure", "microsoft", "cloud", "storage", "queue", "enqueue" })
+@CapabilityDescription("Writes the content of the incoming FlowFiles to the configured Azure Queue Storage.")
+public class PutAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor TTL = new PropertyDescriptor.Builder()
+            .name("time-to-live")
+            .displayName("TTL")
+            .description("Maximum time to allow the message to be in the queue. If left empty, the default value of 7 days will be used.")
+            .required(false)
+            .defaultValue("7 days")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The length of time during which the message will be invisible after it is read. " +
+                    "If the processing unit fails to delete the message after it is read, then the message will reappear in the queue. " +
+                    "If left empty, the default value of 30 secs will be used.")
+            .required(false)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    TTL,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/put-message
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        final int ttl = validationContext.getProperty(TTL).asTimePeriod(TimeUnit.SECONDS).intValue();
+        if (ttl <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .subject(TTL.getDisplayName())
+                    .valid(false)
+                    .explanation(TTL.getDisplayName() + " should be any positive number")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final long startNanos = System.nanoTime();
+
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        session.exportTo(flowFile, baos);
+        final String flowFileContent = baos.toString();
+
+        final int visibilityTimeoutInSecs = context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int ttl = context.getProperty(TTL).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int requestTimeoutInSecs = context.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        final QueueClient queueClient;
+        queueClient = createQueueClient(context, flowFile);

Review Comment:
   The declaration and assignment can be collapsed.
   ```suggestion
           final QueueClient queueClient = createQueueClient(context, flowFile);
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/PutAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueStorageException;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+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.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.ByteArrayOutputStream;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({GetAzureQueueStorage_v12.class})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({ "azure", "microsoft", "cloud", "storage", "queue", "enqueue" })
+@CapabilityDescription("Writes the content of the incoming FlowFiles to the configured Azure Queue Storage.")
+public class PutAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor TTL = new PropertyDescriptor.Builder()
+            .name("time-to-live")
+            .displayName("TTL")
+            .description("Maximum time to allow the message to be in the queue. If left empty, the default value of 7 days will be used.")
+            .required(false)
+            .defaultValue("7 days")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The length of time during which the message will be invisible after it is read. " +
+                    "If the processing unit fails to delete the message after it is read, then the message will reappear in the queue. " +
+                    "If left empty, the default value of 30 secs will be used.")
+            .required(false)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    TTL,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/put-message
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        final int ttl = validationContext.getProperty(TTL).asTimePeriod(TimeUnit.SECONDS).intValue();
+        if (ttl <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .subject(TTL.getDisplayName())
+                    .valid(false)
+                    .explanation(TTL.getDisplayName() + " should be any positive number")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final long startNanos = System.nanoTime();
+
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        session.exportTo(flowFile, baos);
+        final String flowFileContent = baos.toString();
+
+        final int visibilityTimeoutInSecs = context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int ttl = context.getProperty(TTL).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int requestTimeoutInSecs = context.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        final QueueClient queueClient;
+        queueClient = createQueueClient(context, flowFile);
+        try {
+            queueClient.sendMessageWithResponse(
+                    flowFileContent,
+                    Duration.ofSeconds(visibilityTimeoutInSecs),
+                    Duration.ofSeconds(ttl),
+                    Duration.ofSeconds(requestTimeoutInSecs),
+                    Context.NONE
+            );
+        } catch (final QueueStorageException e) {
+            getLogger().error("Failed to write the message to Azure Queue Storage due to {}", new Object[]{e});

Review Comment:
   The wrapping `new Object[]` is not necessary. The exception does not need to be parameterized in the log.
   ```suggestion
               getLogger().error("Failed to write message to Azure Queue Storage", e);
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/AbstractAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.queue;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.ProxyOptions;
+import com.azure.core.http.netty.NettyAsyncHttpClientBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.QueueClientBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AzureServiceEndpoints;
+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 reactor.core.publisher.Mono;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractAzureQueueStorage_v12 extends AbstractProcessor {
+    public static final PropertyDescriptor QUEUE = new PropertyDescriptor.Builder()
+            .name("storage-queue-name")
+            .displayName("Queue Name")
+            .description("Name of the Azure Storage Queue")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor REQUEST_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("request-timeout")
+            .displayName("Request Timeout")
+            .description("The timeout for read or write requests to Azure Queue Storage. " +
+                    "Defaults to 1 second.")
+            .required(true)
+            .defaultValue("1 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("All successfully processed FlowFiles are routed to this relationship")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Unsuccessful operations will be transferred to the failure relationship.")
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE)));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+        final int requestTimeout = validationContext.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (requestTimeout <= 0 || requestTimeout > 30) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(REQUEST_TIMEOUT.getDisplayName())
+                    .explanation(REQUEST_TIMEOUT.getDisplayName() + " should be greater than 0 secs " +
+                            "and less than or equal to 30 secs")
+                    .build());
+        }

Review Comment:
   Is this check necessary? It seems like the Time Period Validator is sufficient without this check.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")
+            .displayName("Batch Size")
+            .description("The number of messages to be retrieved from the queue.")
+            .required(true)
+            .addValidator(StandardValidators.createLongValidator(1, 32, true))
+            .defaultValue("32")
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The duration during which the retrieved message should be invisible to other consumers.")
+            .required(true)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    AUTO_DELETE,
+                    BATCH_SIZE,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/get-messages
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;

Review Comment:
   This value should be created statically and could be simplified using `Duration.ofDays()`.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")
+            .displayName("Batch Size")
+            .description("The number of messages to be retrieved from the queue.")
+            .required(true)
+            .addValidator(StandardValidators.createLongValidator(1, 32, true))
+            .defaultValue("32")
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The duration during which the retrieved message should be invisible to other consumers.")
+            .required(true)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    AUTO_DELETE,
+                    BATCH_SIZE,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/get-messages
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+        final int visibilityTimeoutInSecs = context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int requestTimeoutInSecs = context.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final boolean autoDelete = context.getProperty(AUTO_DELETE).asBoolean();
+
+        final QueueClient queueClient;
+        queueClient = createQueueClient(context, null);
+
+        final Iterable<QueueMessageItem> retrievedMessagesIterable;
+        try {
+            retrievedMessagesIterable = queueClient.receiveMessages(
+                    batchSize,
+                    Duration.ofSeconds(visibilityTimeoutInSecs),
+                    Duration.ofSeconds(requestTimeoutInSecs),
+                    Context.NONE);
+        } catch (final QueueStorageException e) {
+            getLogger().error("Failed to retrieve messages from the provided Azure Storage Queue due to {}", new Object[] {e});
+            context.yield();
+            return;
+        }
+
+        // Map used to store messages to delete after get operation is successful and
+        // auto-delete is enabled (key: messageID, value: popReceipt)
+        final Map<String, String> messagesToDelete = new HashMap<>();
+
+        for (final QueueMessageItem message : retrievedMessagesIterable) {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attributes = new HashMap<>();

Review Comment:
   ```suggestion
               final Map<String, String> attributes = new LinkedHashMap<>();
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")
+            .displayName("Batch Size")
+            .description("The number of messages to be retrieved from the queue.")
+            .required(true)
+            .addValidator(StandardValidators.createLongValidator(1, 32, true))
+            .defaultValue("32")
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The duration during which the retrieved message should be invisible to other consumers.")
+            .required(true)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    AUTO_DELETE,
+                    BATCH_SIZE,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/get-messages
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+        final int visibilityTimeoutInSecs = context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int requestTimeoutInSecs = context.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final boolean autoDelete = context.getProperty(AUTO_DELETE).asBoolean();
+
+        final QueueClient queueClient;
+        queueClient = createQueueClient(context, null);
+
+        final Iterable<QueueMessageItem> retrievedMessagesIterable;
+        try {
+            retrievedMessagesIterable = queueClient.receiveMessages(
+                    batchSize,
+                    Duration.ofSeconds(visibilityTimeoutInSecs),
+                    Duration.ofSeconds(requestTimeoutInSecs),
+                    Context.NONE);
+        } catch (final QueueStorageException e) {
+            getLogger().error("Failed to retrieve messages from the provided Azure Storage Queue due to {}", new Object[] {e});

Review Comment:
   ```suggestion
               getLogger().error("Failed to retrieve messages from Azure Storage Queue", e);
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/queue/TestGetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.queue;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestGetAzureQueueStorage_v12 extends AbstractTestAzureQueueStorage_v12 {
+    @BeforeEach
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(GetAzureQueueStorage_v12.class);
+        setupStorageCredentialsService();
+        runner.enableControllerService(credentialsService);
+        runner.setProperty(GetAzureQueueStorage_v12.STORAGE_CREDENTIALS_SERVICE, CREDENTIALS_SERVICE_IDENTIFIER);
+    }
+
+    @Test
+    public void testValidVisibilityTimeout() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "10 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutZeroSecs() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "0 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("0 secs"));
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutMoreThanSevenDays() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "8 days");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("7 days"));
+    }
+
+    @Test
+    public void testValidRequestTimeout() {
+        runner.setProperty(GetAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(GetAzureQueueStorage_v12.BATCH_SIZE, "10");
+        runner.setProperty(GetAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "10 secs");
+        runner.setProperty(GetAzureQueueStorage_v12.REQUEST_TIMEOUT, "15 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }

Review Comment:
   The `TestRunner.assertValid()` method can be used instead of this approach.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/AbstractAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.queue;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.ProxyOptions;
+import com.azure.core.http.netty.NettyAsyncHttpClientBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.QueueClientBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AzureServiceEndpoints;
+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 reactor.core.publisher.Mono;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractAzureQueueStorage_v12 extends AbstractProcessor {
+    public static final PropertyDescriptor QUEUE = new PropertyDescriptor.Builder()
+            .name("storage-queue-name")
+            .displayName("Queue Name")
+            .description("Name of the Azure Storage Queue")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")

Review Comment:
   ```suggestion
               .name("Credentials Service")
               .displayName("Credentials Service")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/queue/TestPutAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.queue;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestPutAzureQueueStorage_v12 extends AbstractTestAzureQueueStorage_v12 {
+    @BeforeEach
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(PutAzureQueueStorage_v12.class);
+        setupStorageCredentialsService();
+        runner.enableControllerService(credentialsService);
+        runner.setProperty(PutAzureQueueStorage_v12.STORAGE_CREDENTIALS_SERVICE, CREDENTIALS_SERVICE_IDENTIFIER);
+    }
+
+    @Test
+    public void testValidVisibilityTimeout() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "10 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutZeroSecs() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "0 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("0 secs"));
+    }
+
+    @Test
+    public void testInvalidVisibilityTimeoutMoreThanSevenDays() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "8 days");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("7 days"));
+    }
+
+    @Test
+    public void testValidTTL() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "30 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testInvalidTTLZeroSecs() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "0 secs");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "30 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("positive number"));
+    }
+
+    @Test
+    public void testValidRequestTimeout() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "30 secs");
+        runner.setProperty(PutAzureQueueStorage_v12.REQUEST_TIMEOUT, "15 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testInvalidRequestTimeoutZeroSecs() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "30 secs");
+        runner.setProperty(PutAzureQueueStorage_v12.REQUEST_TIMEOUT, "0 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("0 secs"));
+    }
+
+    @Test
+    public void testInvalidRequestTimeoutMoreThanThirtySecs() {
+        runner.setProperty(PutAzureQueueStorage_v12.QUEUE, "dummyqueue");
+        runner.setProperty(PutAzureQueueStorage_v12.TTL, "7 days");
+        runner.setProperty(PutAzureQueueStorage_v12.VISIBILITY_TIMEOUT, "30 secs");
+        runner.setProperty(PutAzureQueueStorage_v12.REQUEST_TIMEOUT, "31 secs");
+
+        ProcessContext processContext = runner.getProcessContext();
+        Collection<ValidationResult> results = new HashSet<>();
+        if (processContext instanceof MockProcessContext) {
+            results = ((MockProcessContext) processContext).validate();
+        }
+
+        assertEquals(1, results.size());
+        Iterator<ValidationResult> iterator = results.iterator();
+        assertTrue(iterator.next().getExplanation().contains("30 secs"));

Review Comment:
   This message check is probably not needed, just checking validity is sufficient.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/AbstractAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.queue;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.ProxyOptions;
+import com.azure.core.http.netty.NettyAsyncHttpClientBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.QueueClientBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AzureServiceEndpoints;
+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 reactor.core.publisher.Mono;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractAzureQueueStorage_v12 extends AbstractProcessor {
+    public static final PropertyDescriptor QUEUE = new PropertyDescriptor.Builder()
+            .name("storage-queue-name")

Review Comment:
   For these new properties, the `name` can match the `displayName`:
   ```suggestion
               .name("Queue Name")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")

Review Comment:
   Recommend adjusting the name to include `Message` for clarity.
   ```suggestion
               .name("Message Batch Size")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/AbstractAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.queue;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.ProxyOptions;
+import com.azure.core.http.netty.NettyAsyncHttpClientBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.QueueClientBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AzureServiceEndpoints;
+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 reactor.core.publisher.Mono;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractAzureQueueStorage_v12 extends AbstractProcessor {
+    public static final PropertyDescriptor QUEUE = new PropertyDescriptor.Builder()
+            .name("storage-queue-name")
+            .displayName("Queue Name")
+            .description("Name of the Azure Storage Queue")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor REQUEST_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("request-timeout")

Review Comment:
   ```suggestion
               .name("Request Timeout")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")
+            .displayName("Batch Size")
+            .description("The number of messages to be retrieved from the queue.")
+            .required(true)
+            .addValidator(StandardValidators.createLongValidator(1, 32, true))
+            .defaultValue("32")
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The duration during which the retrieved message should be invisible to other consumers.")
+            .required(true)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    AUTO_DELETE,
+                    BATCH_SIZE,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/get-messages
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);

Review Comment:
   This should be defined as a static member.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/AbstractAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.queue;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.credential.TokenCredential;
+import com.azure.core.http.HttpClient;
+import com.azure.core.http.ProxyOptions;
+import com.azure.core.http.netty.NettyAsyncHttpClientBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.QueueClientBuilder;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.azure.AzureServiceEndpoints;
+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 reactor.core.publisher.Mono;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractAzureQueueStorage_v12 extends AbstractProcessor {
+    public static final PropertyDescriptor QUEUE = new PropertyDescriptor.Builder()
+            .name("storage-queue-name")
+            .displayName("Queue Name")
+            .description("Name of the Azure Storage Queue")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor STORAGE_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
+            .name("storage-credentials-service")
+            .displayName("Storage Credentials")
+            .description("Controller Service used to obtain Azure Storage Credentials.")
+            .identifiesControllerService(AzureStorageCredentialsService_v12.class)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor REQUEST_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("request-timeout")
+            .displayName("Request Timeout")
+            .description("The timeout for read or write requests to Azure Queue Storage. " +
+                    "Defaults to 1 second.")
+            .required(true)
+            .defaultValue("1 secs")

Review Comment:
   This default value seems low, recommend something higher like 10 seconds.
   ```suggestion
               .defaultValue("10 s")
   ```



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/PutAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueStorageException;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+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.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.ByteArrayOutputStream;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({GetAzureQueueStorage_v12.class})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({ "azure", "microsoft", "cloud", "storage", "queue", "enqueue" })
+@CapabilityDescription("Writes the content of the incoming FlowFiles to the configured Azure Queue Storage.")
+public class PutAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor TTL = new PropertyDescriptor.Builder()
+            .name("time-to-live")
+            .displayName("TTL")
+            .description("Maximum time to allow the message to be in the queue. If left empty, the default value of 7 days will be used.")
+            .required(false)
+            .defaultValue("7 days")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The length of time during which the message will be invisible after it is read. " +
+                    "If the processing unit fails to delete the message after it is read, then the message will reappear in the queue. " +
+                    "If left empty, the default value of 30 secs will be used.")
+            .required(false)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    TTL,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/put-message
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;

Review Comment:
   See note on statically defining this value.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/GetAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueMessageItem;
+import com.azure.storage.queue.models.QueueStorageException;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({PutAzureQueueStorage.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"azure", "queue", "microsoft", "storage", "dequeue", "cloud"})
+@CapabilityDescription("Retrieves the messages from an Azure Queue Storage. The retrieved messages will be deleted from the queue by default. If the requirement is " +
+        "to consume messages without deleting them, set 'Auto Delete Messages' to 'false'. Note: There might be chances of receiving duplicates in situations like " +
+        "when a message is received but was unable to be deleted from the queue due to some unexpected situations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.queue.uri", description = "The absolute URI of the configured Azure Queue Storage"),
+        @WritesAttribute(attribute = "azure.queue.insertionTime", description = "The time when the message was inserted into the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.expirationTime", description = "The time when the message will expire from the queue storage"),
+        @WritesAttribute(attribute = "azure.queue.messageId", description = "The ID of the retrieved message"),
+        @WritesAttribute(attribute = "azure.queue.popReceipt", description = "The pop receipt of the retrieved message"),
+})
+public class GetAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder()
+            .name("auto-delete-messages")
+            .displayName("Auto Delete Messages")
+            .description("Specifies whether the received message is to be automatically deleted from the queue.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("batch-size")
+            .displayName("Batch Size")
+            .description("The number of messages to be retrieved from the queue.")
+            .required(true)
+            .addValidator(StandardValidators.createLongValidator(1, 32, true))
+            .defaultValue("32")
+            .build();
+
+    public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("visibility-timeout")
+            .displayName("Visibility Timeout")
+            .description("The duration during which the retrieved message should be invisible to other consumers.")
+            .required(true)
+            .defaultValue("30 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP, ProxySpec.SOCKS};
+    private static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    QUEUE,
+                    STORAGE_CREDENTIALS_SERVICE,
+                    AUTO_DELETE,
+                    BATCH_SIZE,
+                    VISIBILITY_TIMEOUT,
+                    REQUEST_TIMEOUT,
+                    ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+            )
+    );
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> results = (List<ValidationResult>) super.customValidate(validationContext);
+
+        final int visibilityTimeout = validationContext.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+
+        if (visibilityTimeout <= 0) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should be greater than 0 secs")
+                    .build());
+        }
+
+        // 7 days is the maximum timeout as per https://learn.microsoft.com/en-us/rest/api/storageservices/get-messages
+        final int maxVisibilityTimeout = 7 * 24 * 60 * 60;
+        if (visibilityTimeout >  maxVisibilityTimeout) {
+            results.add(new ValidationResult.Builder()
+                    .valid(false)
+                    .subject(VISIBILITY_TIMEOUT.getDisplayName())
+                    .explanation(VISIBILITY_TIMEOUT.getDisplayName() + " should not be greater than 7 days")
+                    .build());
+        }
+
+        return results;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+        final int visibilityTimeoutInSecs = context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final int requestTimeoutInSecs = context.getProperty(REQUEST_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
+        final boolean autoDelete = context.getProperty(AUTO_DELETE).asBoolean();
+
+        final QueueClient queueClient;
+        queueClient = createQueueClient(context, null);
+
+        final Iterable<QueueMessageItem> retrievedMessagesIterable;
+        try {
+            retrievedMessagesIterable = queueClient.receiveMessages(
+                    batchSize,
+                    Duration.ofSeconds(visibilityTimeoutInSecs),
+                    Duration.ofSeconds(requestTimeoutInSecs),
+                    Context.NONE);
+        } catch (final QueueStorageException e) {
+            getLogger().error("Failed to retrieve messages from the provided Azure Storage Queue due to {}", new Object[] {e});
+            context.yield();
+            return;
+        }
+
+        // Map used to store messages to delete after get operation is successful and
+        // auto-delete is enabled (key: messageID, value: popReceipt)
+        final Map<String, String> messagesToDelete = new HashMap<>();
+
+        for (final QueueMessageItem message : retrievedMessagesIterable) {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put("azure.queue.uri", queueClient.getQueueUrl());
+            attributes.put("azure.queue.insertionTime", message.getInsertionTime().toString());
+            attributes.put("azure.queue.expirationTime", message.getExpirationTime().toString());
+            attributes.put("azure.queue.messageId", message.getMessageId());
+            attributes.put("azure.queue.popReceipt", message.getPopReceipt());

Review Comment:
   Recommend defining static values for this attribute names and reusing in the Processor annotations.



##########
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/queue/PutAzureQueueStorage_v12.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.queue;
+
+import com.azure.core.util.Context;
+import com.azure.storage.queue.QueueClient;
+import com.azure.storage.queue.models.QueueStorageException;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+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.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.ByteArrayOutputStream;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@SeeAlso({GetAzureQueueStorage_v12.class})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({ "azure", "microsoft", "cloud", "storage", "queue", "enqueue" })
+@CapabilityDescription("Writes the content of the incoming FlowFiles to the configured Azure Queue Storage.")
+public class PutAzureQueueStorage_v12 extends AbstractAzureQueueStorage_v12 {
+    public static final PropertyDescriptor TTL = new PropertyDescriptor.Builder()
+            .name("time-to-live")
+            .displayName("TTL")

Review Comment:
   Recommend spelling out this property for improved readability.
   ```suggestion
               .name("Message Time To Live")
               .displayName("Message Time To Live")
   ```



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