You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jg...@apache.org on 2023/10/11 00:33:02 UTC

[nifi] branch main updated: NIFI-12144: Eliminated deprecation warnings by moving to newer Amazon SDK APIs. Eliminated several abstract classes that provided no value. Code cleanup. Refactored many of the integration tests to make use of Test Containers. Removed some tests that were incorrect

This is an automated email from the ASF dual-hosted git repository.

jgresock pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 0fd4ec50ad NIFI-12144: Eliminated deprecation warnings by moving to newer Amazon SDK APIs. Eliminated several abstract classes that provided no value. Code cleanup. Refactored many of the integration tests to make use of Test Containers. Removed some tests that were incorrect
0fd4ec50ad is described below

commit 0fd4ec50adc13bb3458d7349cf73da98606c55b0
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed Oct 4 17:00:38 2023 -0400

    NIFI-12144: Eliminated deprecation warnings by moving to newer Amazon SDK APIs. Eliminated several abstract classes that provided no value. Code cleanup. Refactored many of the integration tests to make use of Test Containers. Removed some tests that were incorrect
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    This closes #7854.
---
 .../nifi-aws-abstract-processors/pom.xml           |   5 +
 .../AbstractAWSCredentialsProviderProcessor.java   | 369 ++++++++++-
 .../nifi/processors/aws/AbstractAWSProcessor.java  | 449 --------------
 .../apache/nifi/processors/aws/AwsClientCache.java |  41 --
 .../nifi/processors/aws/AwsClientDetails.java      |  58 --
 .../nifi/processors/aws/AwsClientProvider.java     |  32 -
 .../processors/aws/AwsPropertyDescriptors.java     |  52 --
 .../factory/CredentialPropertyDescriptors.java     |  20 +-
 .../aws/dynamodb/AbstractDynamoDBProcessor.java    |  60 +-
 .../dynamodb/AbstractWriteDynamoDBProcessor.java   |  66 --
 .../nifi/processors/aws/dynamodb/ItemKeys.java     |  31 +-
 .../aws/kinesis/KinesisProcessorUtils.java         |  20 +-
 .../firehose/AbstractKinesisFirehoseProcessor.java |  62 --
 .../AbstractKinesisStreamAsyncProcessor.java       |  34 -
 .../stream/AbstractKinesisStreamSyncProcessor.java |  34 -
 .../aws/kinesis/stream/KinesisStreamProcessor.java |  31 -
 .../aws/lambda/AbstractAWSLambdaProcessor.java     |  73 ---
 .../processors/aws/s3/AbstractS3Processor.java     | 135 ++--
 .../processors/aws/sns/AbstractSNSProcessor.java   |  57 --
 .../processors/aws/sqs/AbstractSQSProcessor.java   |  50 --
 .../aws/v2/AbstractAwsAsyncProcessor.java          |   7 +-
 .../processors/aws/v2/AbstractAwsProcessor.java    |  55 +-
 .../aws/v2/AbstractAwsSyncProcessor.java           |   8 +-
 .../nifi/processors/aws/v2/AwsClientCache.java     |  42 --
 .../nifi/processors/aws/v2/AwsClientDetails.java   |  59 --
 .../nifi/processors/aws/v2/AwsClientProvider.java  |  31 -
 .../aws/wag/AbstractAWSGatewayApiProcessor.java    | 200 +++---
 .../aws/wag/client/GenericApiGatewayClient.java    |  42 +-
 .../aws/AwsSecretsManagerParameterProvider.java    |   9 +-
 .../nifi-aws-bundle/nifi-aws-processors/pom.xml    |   6 +
 .../AccessKeyPairCredentialsStrategy.java          |  10 +-
 .../strategies/AnonymousCredentialsStrategy.java   |   4 +-
 .../strategies/AssumeRoleCredentialsStrategy.java  |  24 +-
 .../AWSCredentialsProviderControllerService.java   |   4 +-
 .../processors/aws/dynamodb/DeleteDynamoDB.java    |  52 +-
 .../nifi/processors/aws/dynamodb/PutDynamoDB.java  |  61 +-
 .../aws/kinesis/firehose/PutKinesisFirehose.java   | 117 ++--
 .../aws/kinesis/stream/ConsumeKinesisStream.java   |  72 ++-
 .../aws/kinesis/stream/PutKinesisStream.java       |  40 +-
 .../nifi/processors/aws/lambda/PutLambda.java      | 186 +++---
 .../aws/ml/AwsMachineLearningJobStarter.java       |  38 +-
 .../ml/AwsMachineLearningJobStatusProcessor.java   |  29 +-
 .../aws/ml/polly/GetAwsPollyJobStatus.java         |  15 +-
 .../processors/aws/ml/polly/StartAwsPollyJob.java  |  12 +-
 .../aws/ml/textract/GetAwsTextractJobStatus.java   |  59 +-
 .../aws/ml/textract/StartAwsTextractJob.java       |  85 +--
 .../ml/transcribe/GetAwsTranscribeJobStatus.java   |   8 +-
 .../aws/ml/transcribe/StartAwsTranscribeJob.java   |  14 +-
 .../aws/ml/translate/GetAwsTranslateJobStatus.java |  10 +-
 .../aws/ml/translate/StartAwsTranslateJob.java     |   8 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java     |   8 +-
 .../nifi/processors/aws/s3/FetchS3Object.java      |  14 +-
 .../org/apache/nifi/processors/aws/s3/ListS3.java  |  18 +-
 .../apache/nifi/processors/aws/s3/PutS3Object.java | 686 ++++++++++-----------
 .../apache/nifi/processors/aws/s3/TagS3Object.java |  15 +-
 .../encryption/ClientSideCEncryptionStrategy.java  |  35 +-
 .../ClientSideKMSEncryptionStrategy.java           |  61 --
 .../aws/s3/encryption/S3EncryptionStrategy.java    |  12 +-
 .../encryption/ServerSideCEncryptionStrategy.java  |   3 +-
 .../ServerSideKMSEncryptionStrategy.java           |   2 +-
 .../encryption/ServerSideS3EncryptionStrategy.java |   2 +-
 .../s3/encryption/StandardS3EncryptionService.java |  38 +-
 .../org/apache/nifi/processors/aws/sns/PutSNS.java |  69 ++-
 .../apache/nifi/processors/aws/sqs/DeleteSQS.java  |  36 +-
 .../org/apache/nifi/processors/aws/sqs/GetSQS.java |  53 +-
 .../org/apache/nifi/processors/aws/sqs/PutSQS.java |  47 +-
 .../processors/aws/wag/InvokeAWSGatewayApi.java    |  38 +-
 .../additionalDetails.html                         |  20 +
 .../nifi/processors/aws/TestAWSCredentials.java    | 138 -----
 .../nifi/processors/aws/TestAwsClientCache.java    | 100 ---
 .../nifi/processors/aws/TestAwsClientDetails.java  |  49 --
 .../provider/factory/MockAWSProcessor.java         |  26 +-
 .../factory/TestCredentialsProviderFactory.java    |  11 +-
 ...WSCredentialsProviderControllerServiceTest.java |  48 +-
 .../provider/service/AWSProcessorProxyTest.java    |  63 +-
 .../aws/dynamodb/AbstractDynamoDBTest.java         |  23 +
 .../aws/dynamodb/DeleteDynamoDBTest.java           |   4 +-
 .../processors/aws/dynamodb/GetDynamoDBTest.java   |   5 +-
 .../aws/dynamodb/ITAbstractDynamoDBTest.java       | 138 -----
 .../processors/aws/dynamodb/PutDynamoDBTest.java   |  56 +-
 .../aws/kinesis/firehose/ITPutKinesisFirehose.java | 281 +++------
 .../kinesis/firehose/TestPutKinesisFirehose.java   |   8 +-
 .../kinesis/stream/TestConsumeKinesisStream.java   |  15 +-
 .../aws/ml/polly/GetAwsPollyStatusTest.java        |  28 +-
 .../ml/textract/GetAwsTextractJobStatusTest.java   |  23 +-
 .../transcribe/GetAwsTranscribeJobStatusTest.java  |  26 +-
 .../ml/translate/GetAwsTranslateJobStatusTest.java |  26 +-
 .../nifi/processors/aws/s3/AbstractS3IT.java       | 161 +++--
 .../nifi/processors/aws/s3/ITDeleteS3Object.java   |  55 +-
 .../nifi/processors/aws/s3/ITFetchS3Object.java    |  52 +-
 .../apache/nifi/processors/aws/s3/ITListS3.java    |  72 +--
 .../nifi/processors/aws/s3/ITPutS3Object.java      | 444 +++----------
 .../nifi/processors/aws/s3/ITTagS3Object.java      |  94 ++-
 .../nifi/processors/aws/s3/TestDeleteS3Object.java |  45 +-
 .../nifi/processors/aws/s3/TestFetchS3Object.java  |  25 +-
 .../apache/nifi/processors/aws/s3/TestListS3.java  |  31 +-
 .../nifi/processors/aws/s3/TestPutS3Object.java    |  17 +-
 .../nifi/processors/aws/s3/TestTagS3Object.java    |  28 +-
 .../s3/encryption/TestS3EncryptionStrategies.java  |  36 +-
 .../TestStandardS3EncryptionService.java           |   2 +-
 .../nifi/processors/aws/sqs/AbstractSQSIT.java     | 108 ++++
 .../nifi/processors/aws/sqs/ITDeleteSQS.java       |  46 +-
 .../apache/nifi/processors/aws/sqs/ITGetSQS.java   |  90 +--
 .../apache/nifi/processors/aws/sqs/ITPutSQS.java   |  54 +-
 .../aws/wag/TestInvokeAWSGatewayApiCommon.java     |   5 +-
 ...wayApi.java => TestInvokeAmazonGatewayApi.java} |   2 +-
 .../aws/wag/TestInvokeAmazonGatewayApiMock.java    |   2 +-
 .../aws/s3/AmazonS3EncryptionService.java          |  13 +-
 108 files changed, 2470 insertions(+), 4123 deletions(-)

diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
index 8b222b5153..95bef9c10f 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/pom.xml
@@ -131,6 +131,11 @@
             <artifactId>caffeine</artifactId>
             <version>2.9.2</version>
         </dependency>
+        <!-- Version 2 of the AmazonS3EncryptionClient requires bouncy castle -->
+        <dependency>
+            <groupId>org.bouncycastle</groupId>
+            <artifactId>bcprov-jdk18on</artifactId>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSCredentialsProviderProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSCredentialsProviderProcessor.java
index 11a5b8c276..4999ee100b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSCredentialsProviderProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSCredentialsProviderProcessor.java
@@ -18,19 +18,55 @@ package org.apache.nifi.processors.aws;
 
 import com.amazonaws.AmazonWebServiceClient;
 import com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
+import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.PropertiesCredentials;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.http.conn.ssl.SdkTLSSocketFactory;
+import com.amazonaws.regions.Region;
+import com.amazonaws.regions.Regions;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.http.conn.ssl.DefaultHostnameVerifier;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.ConfigVerificationResult.Outcome;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.components.PropertyValue;
+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.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxyConfigurationService;
+import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.ssl.SSLContextService;
 
+import javax.net.ssl.SSLContext;
+import java.io.File;
+import java.io.IOException;
+import java.net.Proxy;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Base class for AWS processors that uses AWSCredentialsProvider interface for creating AWS clients.
@@ -39,14 +75,82 @@ import java.util.Map;
  *
  * @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
  */
-public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends AmazonWebServiceClient>
-    extends AbstractAWSProcessor<ClientType> implements VerifiableProcessor {
+public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends AmazonWebServiceClient> extends AbstractProcessor
+        implements VerifiableProcessor {
+
+    // Property Descriptors
+    public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
+    public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY_ID;
+    public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
+
+    public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
+            .name("Proxy Host")
+            .description("Proxy host name or IP")
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PROXY_HOST_PORT = new PropertyDescriptor.Builder()
+            .name("Proxy Host Port")
+            .description("Proxy host port")
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(false)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PROXY_USERNAME = new PropertyDescriptor.Builder()
+            .name("proxy-user-name")
+            .displayName("Proxy Username")
+            .description("Proxy username")
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PROXY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("proxy-user-password")
+            .displayName("Proxy Password")
+            .description("Proxy password")
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .name("Region")
+            .description("The AWS Region to connect to.")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+
+    public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
+            .name("Communications Timeout")
+            .description("The amount of time to wait in order to establish a connection to AWS or receive data from AWS before timing out.")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description("Specifies an optional SSL Context Service that, if provided, will be used to create connections")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    public static final PropertyDescriptor ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .name("Endpoint Override URL")
+            .description("Endpoint URL to use instead of the AWS default including scheme, host, port, and path. " +
+                    "The AWS libraries select an endpoint URL based on the AWS region, but this property overrides " +
+                    "the selected endpoint URL, allowing use with other S3-compatible endpoints.")
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(false)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .build();
 
-    /**
-     * AWS credentials provider service
-     *
-     * @see  <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
-     */
     public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
             .name("AWS Credentials Provider service")
             .displayName("AWS Credentials Provider Service")
@@ -55,34 +159,222 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
             .identifiesControllerService(AWSCredentialsProviderService.class)
             .build();
 
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles are routed to this Relationship after they have been successfully processed.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If the Processor is unable to process a given FlowFile, it will be routed to this Relationship.")
+            .build();
+
+    public static final Set<Relationship> relationships = Set.of(REL_SUCCESS, REL_FAILURE);
+
+
+    // Constants
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
+
+
+    // Member variables
+    private final Cache<String, ClientType> clientCache = Caffeine.newBuilder()
+            .maximumSize(10)
+            .build();
+
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        getClient(context);
+    }
+
+    @OnStopped
+    public void onStopped() {
+        this.clientCache.invalidateAll();
+        this.clientCache.cleanUp();
+    }
+
+
+    public static AllowableValue createAllowableValue(final Regions region) {
+        return new AllowableValue(region.getName(), region.getDescription(), "AWS Region Code : " + region.getName());
+    }
+
+    public static AllowableValue[] getAvailableRegions() {
+        final List<AllowableValue> values = new ArrayList<>();
+        for (final Regions region : Regions.values()) {
+            values.add(createAllowableValue(region));
+        }
+        return values.toArray(new AllowableValue[0]);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
+
+        final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet();
+        final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet();
+        if ((accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet)) {
+            validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("If setting Secret Key or Access Key, must set both").build());
+        }
+
+        final boolean credentialsFileSet = validationContext.getProperty(CREDENTIALS_FILE).isSet();
+        if ((secretKeySet || accessKeySet) && credentialsFileSet) {
+            validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("Cannot set both Credentials File and Secret Key/Access Key").build());
+        }
+
+        final boolean proxyHostSet = validationContext.getProperty(PROXY_HOST).isSet();
+        final boolean proxyPortSet = validationContext.getProperty(PROXY_HOST_PORT).isSet();
+        final boolean proxyConfigServiceSet = validationContext.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet();
+
+        if ((proxyHostSet && !proxyPortSet) || (!proxyHostSet && proxyPortSet)) {
+            validationResults.add(new ValidationResult.Builder().subject("Proxy Host and Port").valid(false).explanation("If Proxy Host or Proxy Port is set, both must be set").build());
+        }
+
+        final boolean proxyUserSet = validationContext.getProperty(PROXY_USERNAME).isSet();
+        final boolean proxyPwdSet = validationContext.getProperty(PROXY_PASSWORD).isSet();
+
+        if ((proxyUserSet && !proxyPwdSet) || (!proxyUserSet && proxyPwdSet)) {
+            validationResults.add(new ValidationResult.Builder().subject("Proxy User and Password").valid(false).explanation("If Proxy Username or Proxy Password is set, both must be set").build());
+        }
+
+        if (proxyUserSet && !proxyHostSet) {
+            validationResults.add(new ValidationResult.Builder().subject("Proxy").valid(false).explanation("If Proxy Username or Proxy Password").build());
+        }
+
+        ProxyConfiguration.validateProxySpec(validationContext, validationResults, PROXY_SPECS);
+
+        if (proxyHostSet && proxyConfigServiceSet) {
+            validationResults.add(new ValidationResult.Builder().subject("Proxy Configuration Service").valid(false)
+                    .explanation("Either Proxy Username and Proxy Password must be set or Proxy Configuration Service but not both").build());
+        }
+
+        return validationResults;
+    }
+
+
+    protected AWSCredentials getCredentials(final PropertyContext context) {
+        final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
+        final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();
+
+        final String credentialsFile = context.getProperty(CREDENTIALS_FILE).getValue();
+
+        if (credentialsFile != null) {
+            try {
+                return new PropertiesCredentials(new File(credentialsFile));
+            } catch (final IOException ioe) {
+                throw new ProcessException("Could not read Credentials File", ioe);
+            }
+        }
+
+        if (accessKey != null && secretKey != null) {
+            return new BasicAWSCredentials(accessKey, secretKey);
+        }
+
+        return new AnonymousAWSCredentials();
+    }
+
+
+    protected ClientConfiguration createConfiguration(final ProcessContext context) {
+        return createConfiguration(context, context.getMaxConcurrentTasks());
+    }
+
+    protected ClientConfiguration createConfiguration(final PropertyContext context, final int maxConcurrentTasks) {
+        final ClientConfiguration config = new ClientConfiguration();
+        config.setMaxConnections(maxConcurrentTasks);
+        config.setMaxErrorRetry(0);
+        config.setUserAgentPrefix("NiFi");
+        config.setProtocol(Protocol.HTTPS);
+
+        final int commsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
+        config.setConnectionTimeout(commsTimeout);
+        config.setSocketTimeout(commsTimeout);
+
+        if (this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
+            final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+            if (sslContextService != null) {
+                final SSLContext sslContext = sslContextService.createContext();
+                // NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
+                SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
+                config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
+            }
+        }
+
+        final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
+            if (context.getProperty(PROXY_HOST).isSet()) {
+                final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
+                String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
+                Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
+                String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
+                String proxyPassword = context.getProperty(PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
+                componentProxyConfig.setProxyType(Proxy.Type.HTTP);
+                componentProxyConfig.setProxyServerHost(proxyHost);
+                componentProxyConfig.setProxyServerPort(proxyPort);
+                componentProxyConfig.setProxyUserName(proxyUsername);
+                componentProxyConfig.setProxyUserPassword(proxyPassword);
+                return componentProxyConfig;
+            } else if (context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet()) {
+                final ProxyConfigurationService configurationService = context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).asControllerService(ProxyConfigurationService.class);
+                return configurationService.getConfiguration();
+            }
+            return ProxyConfiguration.DIRECT_CONFIGURATION;
+        });
+
+        if (Proxy.Type.HTTP.equals(proxyConfig.getProxyType())) {
+            config.setProxyHost(proxyConfig.getProxyServerHost());
+            config.setProxyPort(proxyConfig.getProxyServerPort());
+
+            if (proxyConfig.hasCredential()) {
+                config.setProxyUsername(proxyConfig.getProxyUserName());
+                config.setProxyPassword(proxyConfig.getProxyUserPassword());
+            }
+        }
+
+        return config;
+    }
+
+
+    protected ClientType createClient(final ProcessContext context) {
+        return createClient(context, getRegion(context));
+    }
+
     /**
      * Attempts to create the client using the controller service first before falling back to the standard configuration.
      * @param context The process context
      * @return The created client
      */
-    @Override
-    public ClientType createClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
-        final ControllerService service = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).asControllerService();
-        if (service != null) {
-            getLogger().debug("Using AWS credentials provider service for creating client");
-            final AWSCredentialsProvider credentialsProvider = getCredentialsProvider(context);
-            final ClientConfiguration configuration = createConfiguration(context);
-            final ClientType createdClient = createClient(context, credentialsProvider, configuration);
-            setRegionAndInitializeEndpoint(awsClientDetails.getRegion(), context, createdClient);
-            return createdClient;
-        } else {
-            getLogger().debug("Using AWS credentials for creating client");
-            return super.createClient(context, awsClientDetails);
+    public ClientType createClient(final ProcessContext context, final Region region) {
+        getLogger().debug("Using AWS credentials provider service for creating client");
+        final AWSCredentialsProvider credentialsProvider = getCredentialsProvider(context);
+        final ClientConfiguration configuration = createConfiguration(context);
+        final ClientType createdClient = createClient(context, credentialsProvider, region, configuration, getEndpointConfiguration(context, region));
+        return createdClient;
+    }
+
+    protected AwsClientBuilder.EndpointConfiguration getEndpointConfiguration(final ProcessContext context, final Region region) {
+        final PropertyValue overrideValue = context.getProperty(ENDPOINT_OVERRIDE);
+        if (overrideValue == null || !overrideValue.isSet()) {
+            return null;
         }
+
+        final String endpointOverride = overrideValue.getValue();
+        return new AwsClientBuilder.EndpointConfiguration(endpointOverride, region.getName());
     }
 
 
+
     @Override
     public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
         final List<ConfigVerificationResult> results = new ArrayList<>();
 
         try {
             createClient(context);
+
             results.add(new ConfigVerificationResult.Builder()
                     .outcome(Outcome.SUCCESSFUL)
                     .verificationStepName("Create Client and Configure Region")
@@ -107,13 +399,42 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
      * @see  <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
      */
     protected AWSCredentialsProvider getCredentialsProvider(final ProcessContext context) {
-
         final AWSCredentialsProviderService awsCredentialsProviderService =
               context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).asControllerService(AWSCredentialsProviderService.class);
 
+        if (awsCredentialsProviderService == null) {
+            final AWSCredentials credentials = getCredentials(context);
+            return new AWSStaticCredentialsProvider(credentials);
+        }
+
         return awsCredentialsProviderService.getCredentialsProvider();
     }
 
+
+    protected Region getRegion(final ProcessContext context) {
+        // if the processor supports REGION, get the configured region.
+        if (getSupportedPropertyDescriptors().contains(REGION)) {
+            final String regionValue = context.getProperty(REGION).getValue();
+            if (regionValue != null) {
+                return Region.getRegion(Regions.fromName(regionValue));
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Creates an AWS service client from the context or returns an existing client from the cache
+     */
+    protected ClientType getClient(final ProcessContext context, final Region region) {
+        final String regionName = region == null ? "" : region.getName();
+        return clientCache.get(regionName, ignored -> createClient(context, region));
+    }
+
+    protected ClientType getClient(final ProcessContext context) {
+        return getClient(context, getRegion(context));
+    }
+
     /**
      * Abstract method to create AWS client using credentials provider. This is the preferred method
      * for creating AWS clients
@@ -122,5 +443,7 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
      * @param config AWS client configuration
      * @return ClientType the client
      */
-    protected abstract ClientType createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config);
+    protected abstract ClientType createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, Region region, ClientConfiguration config,
+                                               AwsClientBuilder.EndpointConfiguration endpointConfiguration);
+
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
deleted file mode 100644
index 95fc1d9d55..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
+++ /dev/null
@@ -1,449 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.AmazonWebServiceClient;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AnonymousAWSCredentials;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.http.conn.ssl.SdkTLSSocketFactory;
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.http.conn.ssl.DefaultHostnameVerifier;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.components.AllowableValue;
-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.processor.AbstractSessionFactoryProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
-import org.apache.nifi.proxy.ProxyConfiguration;
-import org.apache.nifi.proxy.ProxyConfigurationService;
-import org.apache.nifi.proxy.ProxySpec;
-import org.apache.nifi.ssl.SSLContextService;
-
-import javax.net.ssl.SSLContext;
-import java.io.File;
-import java.io.IOException;
-import java.net.Proxy;
-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;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * Abstract base class for AWS processors.  This class uses AWS credentials for creating AWS clients
- *
- * @deprecated use {@link AbstractAWSCredentialsProviderProcessor} instead which uses credentials providers or creating AWS clients
- * @see AbstractAWSCredentialsProviderProcessor
- *
- */
-@Deprecated
-public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceClient> extends AbstractSessionFactoryProcessor implements AwsClientProvider<ClientType> {
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
-            .description("FlowFiles are routed to success relationship").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("FlowFiles are routed to failure relationship").build();
-
-    public static final Set<Relationship> relationships = Collections.unmodifiableSet(
-            new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE)));
-
-    public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
-    public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY;
-    public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
-
-    public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
-            .name("Proxy Host")
-            .description("Proxy host name or IP")
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROXY_HOST_PORT = new PropertyDescriptor.Builder()
-            .name("Proxy Host Port")
-            .description("Proxy host port")
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .required(false)
-            .addValidator(StandardValidators.PORT_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROXY_USERNAME = new PropertyDescriptor.Builder()
-            .name("proxy-user-name")
-            .displayName("Proxy Username")
-            .description("Proxy username")
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROXY_PASSWORD = new PropertyDescriptor.Builder()
-            .name("proxy-user-password")
-            .displayName("Proxy Password")
-            .description("Proxy password")
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .sensitive(true)
-            .build();
-
-    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
-            .name("Region")
-            .description("The AWS Region to connect to.")
-            .required(true)
-            .allowableValues(getAvailableRegions())
-            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
-            .build();
-
-    public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
-            .name("Communications Timeout")
-            .description("The amount of time to wait in order to establish a connection to AWS or receive data from AWS before timing out.")
-            .required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .defaultValue("30 secs")
-            .build();
-
-    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-            .name("SSL Context Service")
-            .description("Specifies an optional SSL Context Service that, if provided, will be used to create connections")
-            .required(false)
-            .identifiesControllerService(SSLContextService.class)
-            .build();
-
-    public static final PropertyDescriptor ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
-            .name("Endpoint Override URL")
-            .description("Endpoint URL to use instead of the AWS default including scheme, host, port, and path. " +
-                    "The AWS libraries select an endpoint URL based on the AWS region, but this property overrides " +
-                    "the selected endpoint URL, allowing use with other S3-compatible endpoints.")
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .required(false)
-            .addValidator(StandardValidators.URL_VALIDATOR)
-            .build();
-
-    protected static final String VPCE_ENDPOINT_SUFFIX = ".vpce.amazonaws.com";
-    protected static final Pattern VPCE_ENDPOINT_PATTERN = Pattern.compile("^(?:.+[vpce-][a-z0-9-]+\\.)?([a-z0-9-]+)$");
-
-    // If protocol is changed to be a property, ensure other uses are also changed
-    protected static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
-    protected static final String DEFAULT_USER_AGENT = "NiFi";
-
-    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
-    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
-
-    private final AwsClientCache<ClientType> awsClientCache = new AwsClientCache<>();
-
-    public static AllowableValue createAllowableValue(final Regions region) {
-        return new AllowableValue(region.getName(), region.getDescription(), "AWS Region Code : " + region.getName());
-    }
-
-    public static AllowableValue[] getAvailableRegions() {
-        final List<AllowableValue> values = new ArrayList<>();
-        for (final Regions region : Regions.values()) {
-            values.add(createAllowableValue(region));
-        }
-        return values.toArray(new AllowableValue[0]);
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
-
-        final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet();
-        final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet();
-        if ((accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet)) {
-            validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("If setting Secret Key or Access Key, must set both").build());
-        }
-
-        final boolean credentialsFileSet = validationContext.getProperty(CREDENTIALS_FILE).isSet();
-        if ((secretKeySet || accessKeySet) && credentialsFileSet) {
-            validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("Cannot set both Credentials File and Secret Key/Access Key").build());
-        }
-
-        final boolean proxyHostSet = validationContext.getProperty(PROXY_HOST).isSet();
-        final boolean proxyPortSet = validationContext.getProperty(PROXY_HOST_PORT).isSet();
-        final boolean proxyConfigServiceSet = validationContext.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet();
-
-        if ((proxyHostSet && !proxyPortSet) || (!proxyHostSet && proxyPortSet)) {
-            validationResults.add(new ValidationResult.Builder().subject("Proxy Host and Port").valid(false).explanation("If Proxy Host or Proxy Port is set, both must be set").build());
-        }
-
-        final boolean proxyUserSet = validationContext.getProperty(PROXY_USERNAME).isSet();
-        final boolean proxyPwdSet = validationContext.getProperty(PROXY_PASSWORD).isSet();
-
-        if ((proxyUserSet && !proxyPwdSet) || (!proxyUserSet && proxyPwdSet)) {
-            validationResults.add(new ValidationResult.Builder().subject("Proxy User and Password").valid(false).explanation("If Proxy Username or Proxy Password is set, both must be set").build());
-        }
-
-        if (proxyUserSet && !proxyHostSet) {
-            validationResults.add(new ValidationResult.Builder().subject("Proxy").valid(false).explanation("If Proxy Username or Proxy Password").build());
-        }
-
-        ProxyConfiguration.validateProxySpec(validationContext, validationResults, PROXY_SPECS);
-
-        if (proxyHostSet && proxyConfigServiceSet) {
-            validationResults.add(new ValidationResult.Builder().subject("Proxy Configuration Service").valid(false)
-                    .explanation("Either Proxy Username and Proxy Password must be set or Proxy Configuration Service but not both").build());
-        }
-
-        return validationResults;
-    }
-    protected ClientConfiguration createConfiguration(final ProcessContext context) {
-        return createConfiguration(context, context.getMaxConcurrentTasks());
-    }
-
-    protected ClientConfiguration createConfiguration(final PropertyContext context, final int maxConcurrentTasks) {
-        final ClientConfiguration config = new ClientConfiguration();
-        config.setMaxConnections(maxConcurrentTasks);
-        config.setMaxErrorRetry(0);
-        config.setUserAgent(DEFAULT_USER_AGENT);
-        // If this is changed to be a property, ensure other uses are also changed
-        config.setProtocol(DEFAULT_PROTOCOL);
-        final int commsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
-        config.setConnectionTimeout(commsTimeout);
-        config.setSocketTimeout(commsTimeout);
-
-        if(this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
-            final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-            if (sslContextService != null) {
-                final SSLContext sslContext = sslContextService.createContext();
-                // NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
-                SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
-                config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
-            }
-        }
-
-        final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
-            if (context.getProperty(PROXY_HOST).isSet()) {
-                final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
-                String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
-                Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
-                String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
-                String proxyPassword = context.getProperty(PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
-                componentProxyConfig.setProxyType(Proxy.Type.HTTP);
-                componentProxyConfig.setProxyServerHost(proxyHost);
-                componentProxyConfig.setProxyServerPort(proxyPort);
-                componentProxyConfig.setProxyUserName(proxyUsername);
-                componentProxyConfig.setProxyUserPassword(proxyPassword);
-                return componentProxyConfig;
-            } else if (context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet()) {
-                final ProxyConfigurationService configurationService = context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).asControllerService(ProxyConfigurationService.class);
-                return configurationService.getConfiguration();
-            }
-            return ProxyConfiguration.DIRECT_CONFIGURATION;
-        });
-
-        if (Proxy.Type.HTTP.equals(proxyConfig.getProxyType())) {
-            config.setProxyHost(proxyConfig.getProxyServerHost());
-            config.setProxyPort(proxyConfig.getProxyServerPort());
-
-            if (proxyConfig.hasCredential()) {
-                config.setProxyUsername(proxyConfig.getProxyUserName());
-                config.setProxyPassword(proxyConfig.getProxyUserPassword());
-            }
-        }
-
-        return config;
-    }
-
-    @OnScheduled
-    public void onScheduled(final ProcessContext context) {
-        getClient(context);
-    }
-
-    /*
-     * Allow optional override of onTrigger with the ProcessSessionFactory where required for AWS processors (e.g. ConsumeKinesisStream)
-     *
-     * @see AbstractProcessor
-     */
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        final ProcessSession session = sessionFactory.createSession();
-        try {
-            onTrigger(context, session);
-            session.commitAsync();
-        } catch (final Throwable t) {
-            session.rollback(true);
-            throw t;
-        }
-    }
-
-    /*
-     * Default to requiring the "standard" onTrigger with a single ProcessSession
-     */
-    public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
-
-    public ClientType createClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
-        final AWSCredentials credentials = getCredentials(context);
-        final ClientConfiguration configuration = createConfiguration(context);
-        final ClientType createdClient = createClient(context, credentials, configuration);
-
-        setRegionAndInitializeEndpoint(awsClientDetails.getRegion(), context, createdClient);
-        return createdClient;
-    }
-
-    protected ClientType createClient(ProcessContext context) {
-        return createClient(context, new AwsClientDetails(getRegion(context)));
-    }
-
-
-    protected void setRegionAndInitializeEndpoint(final Region region, final ProcessContext context, final AmazonWebServiceClient client) {
-        if (region!= null && client != null) {
-            client.setRegion(region);
-        }
-
-        // if the endpoint override has been configured, set the endpoint.
-        // (per Amazon docs this should only be configured at client creation)
-        if (client != null && getSupportedPropertyDescriptors().contains(ENDPOINT_OVERRIDE)) {
-            final String urlstr = StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue());
-
-            if (!urlstr.isEmpty()) {
-                getLogger().info("Overriding endpoint with {}", urlstr);
-
-                if (urlstr.endsWith(VPCE_ENDPOINT_SUFFIX)) {
-                    // handling vpce endpoints
-                    // falling back to the configured region if the parse fails
-                    // e.g. in case of https://vpce-***-***.sqs.{region}.vpce.amazonaws.com
-                    String regionValue = parseRegionForVPCE(urlstr, region.getName());
-                    client.setEndpoint(urlstr, client.getServiceName(), regionValue);
-                } else if (isCustomSignerConfigured(context)) {
-                    // handling endpoints with a user defined custom signer
-                    client.setEndpoint(urlstr, client.getServiceName(), region.getName());
-                } else {
-                    // handling other (non-vpce, no custom signer) custom endpoints where the AWS library can parse the region out
-                    // e.g. https://sqs.{region}.***.***.***.gov
-                    client.setEndpoint(urlstr);
-                }
-            }
-        }
-    }
-
-    protected Region getRegion(final ProcessContext context) {
-        // if the processor supports REGION, get the configured region.
-        if (getSupportedPropertyDescriptors().contains(REGION)) {
-            final String regionValue = context.getProperty(REGION).getValue();
-            if (regionValue != null) {
-                return Region.getRegion(Regions.fromName(regionValue));
-            }
-        }
-        return null;
-    }
-
-    protected boolean isCustomSignerConfigured(final ProcessContext context) {
-        return false;
-    }
-
-    /*
-    Note to developer(s):
-        When setting an endpoint for an AWS Client i.e. client.setEndpoint(endpointUrl),
-        AWS Java SDK fails to parse the region correctly when the provided endpoint
-        is an AWS PrivateLink so this method does the job of parsing the region name and
-        returning it.
-
-        Refer NIFI-5456, NIFI-5893 & NIFI-8662
-     */
-    private String parseRegionForVPCE(String url, String configuredRegion) {
-        int index = url.length() - VPCE_ENDPOINT_SUFFIX.length();
-
-        Matcher matcher = VPCE_ENDPOINT_PATTERN.matcher(url.substring(0, index));
-
-        if (matcher.matches()) {
-            return matcher.group(1);
-        } else {
-            getLogger().info("Unable to get a match with the VPCE endpoint pattern; using the configured region: " + configuredRegion);
-            return configuredRegion;
-        }
-    }
-
-    protected AWSCredentials getCredentials(final PropertyContext context) {
-        final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
-        final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();
-
-        final String credentialsFile = context.getProperty(CREDENTIALS_FILE).getValue();
-
-        if (credentialsFile != null) {
-            try {
-                return new PropertiesCredentials(new File(credentialsFile));
-            } catch (final IOException ioe) {
-                throw new ProcessException("Could not read Credentials File", ioe);
-            }
-        }
-
-        if (accessKey != null && secretKey != null) {
-            return new BasicAWSCredentials(accessKey, secretKey);
-        }
-
-        return new AnonymousAWSCredentials();
-    }
-
-    @OnStopped
-    public void onStopped() {
-        this.awsClientCache.clearCache();
-    }
-
-    /**
-     * Creates an AWS service client from the context or returns an existing client from the cache
-     * @param context The process context
-     * @param  awsClientDetails details of the AWS client
-     * @return The created client
-     */
-    protected ClientType getClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
-        return this.awsClientCache.getOrCreateClient(context, awsClientDetails, this);
-    }
-
-    protected ClientType getClient(final ProcessContext context) {
-        final AwsClientDetails awsClientDetails = new AwsClientDetails(getRegion(context));
-        return getClient(context, awsClientDetails);
-    }
-
-    /**
-     * Create client from the arguments
-     * @param context process context
-     * @param credentials static aws credentials
-     * @param config aws client configuration
-     * @return ClientType aws client
-     *
-     * @deprecated use {@link AbstractAWSCredentialsProviderProcessor#createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)}
-     */
-    @Deprecated
-    protected abstract ClientType createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config);
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientCache.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientCache.java
deleted file mode 100644
index fad4c6d457..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientCache.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.AmazonWebServiceClient;
-import com.github.benmanes.caffeine.cache.Cache;
-import com.github.benmanes.caffeine.cache.Caffeine;
-import org.apache.nifi.processor.ProcessContext;
-
-public class AwsClientCache<ClientType extends AmazonWebServiceClient> {
-
-    private static final int MAXIMUM_CACHE_SIZE = 10;
-
-    private final Cache<AwsClientDetails, ClientType> clientCache = Caffeine.newBuilder()
-            .maximumSize(MAXIMUM_CACHE_SIZE)
-            .build();
-
-    public ClientType getOrCreateClient(final ProcessContext context, final AwsClientDetails clientDetails, final AwsClientProvider<ClientType> provider) {
-        return clientCache.get(clientDetails, ignored -> provider.createClient(context, clientDetails));
-    }
-
-    public void clearCache() {
-        clientCache.invalidateAll();
-        clientCache.cleanUp();
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientDetails.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientDetails.java
deleted file mode 100644
index 22e2a2ce9a..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientDetails.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.regions.Region;
-import java.util.Objects;
-
-/**
- * This class contains the AWS client details used to distinguish between the various AWS clients stored in the cache.
- * The class acts as a cache key for @link AwsClientCache.
- * AwsClientDetails contains the region only, since actually the region value may come from the FlowFile attributes.
- */
-public class AwsClientDetails {
-    private Region region;
-
-    public AwsClientDetails(Region region) {
-       this.region =  region;
-    }
-
-    public Region getRegion() {
-        return region;
-    }
-
-    public void setRegion(final Region region) {
-        this.region = region;
-    }
-
-    @Override
-    public boolean equals(final Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        final AwsClientDetails that = (AwsClientDetails) o;
-        return Objects.equals(region, that.region);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(region);
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientProvider.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientProvider.java
deleted file mode 100644
index 64260495c3..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsClientProvider.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.AmazonWebServiceClient;
-import org.apache.nifi.processor.ProcessContext;
-
-public interface AwsClientProvider<ClientType extends AmazonWebServiceClient> {
-
-    /**
-     * Creates an AWS client using process context and AWS client details.
-     *
-     * @param context process context
-     * @param awsClientDetails AWS client details
-     * @return AWS client
-     */
-    ClientType createClient(final ProcessContext context, final AwsClientDetails awsClientDetails);
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsPropertyDescriptors.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsPropertyDescriptors.java
deleted file mode 100644
index 87fe81debb..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AwsPropertyDescriptors.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.auth.Signer;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.resource.ResourceCardinality;
-import org.apache.nifi.components.resource.ResourceType;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.util.StandardValidators;
-
-public final class AwsPropertyDescriptors {
-
-    private AwsPropertyDescriptors() {
-        // constant class' constructor
-    }
-
-    public static final PropertyDescriptor CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
-            .name("custom-signer-class-name")
-            .displayName("Custom Signer Class Name")
-            .description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .build();
-
-    public static final PropertyDescriptor CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
-            .name("custom-signer-module-location")
-            .displayName("Custom Signer Module Location")
-            .description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
-            .dynamicallyModifiesClasspath(true)
-            .build();
-
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
index 43983b3574..89e2d1c9fc 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/CredentialPropertyDescriptors.java
@@ -16,13 +16,13 @@
  */
 package org.apache.nifi.processors.aws.credentials.provider.factory;
 
+import com.amazonaws.auth.Signer;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.resource.ResourceCardinality;
 import org.apache.nifi.components.resource.ResourceType;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.AwsPropertyDescriptors;
 import org.apache.nifi.ssl.SSLContextService;
 import software.amazon.awssdk.regions.Region;
 
@@ -71,7 +71,7 @@ public class CredentialPropertyDescriptors {
             .description("Path to a file containing AWS access key and secret key in properties file format.")
             .build();
 
-    public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor ACCESS_KEY_ID = new PropertyDescriptor.Builder()
             .name("Access Key")
             .displayName("Access Key ID")
             .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
@@ -245,13 +245,25 @@ public class CredentialPropertyDescriptors {
             .build();
 
     public static final PropertyDescriptor ASSUME_ROLE_STS_CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_CLASS_NAME)
+            .name("custom-signer-class-name")
+            .displayName("Custom Signer Class Name")
+            .description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
             .dependsOn(ASSUME_ROLE_STS_SIGNER_OVERRIDE, CUSTOM_SIGNER)
             .build();
 
     public static final PropertyDescriptor ASSUME_ROLE_STS_CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_MODULE_LOCATION)
+            .name("custom-signer-module-location")
+            .displayName("Custom Signer Module Location")
+            .description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
             .dependsOn(ASSUME_ROLE_STS_SIGNER_OVERRIDE, CUSTOM_SIGNER)
+            .dynamicallyModifiesClasspath(true)
             .build();
 
     public static AllowableValue createAllowableValue(final Region region) {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
index df8b99563c..9c52d1a77d 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
@@ -16,17 +16,15 @@
  */
 package org.apache.nifi.processors.aws.dynamodb;
 
-import java.math.BigDecimal;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.model.AttributeValue;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
@@ -39,14 +37,16 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.model.AttributeValue;
+import java.math.BigDecimal;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Base class for NiFi dynamo db related processors
@@ -177,22 +177,17 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
      * Create client using credentials provider. This is the preferred way for creating clients
      */
     @Override
-    protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+    protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         getLogger().debug("Creating client with credentials provider");
-        return new AmazonDynamoDBClient(credentialsProvider, config);
+        return (AmazonDynamoDBClient) AmazonDynamoDBClient.builder()
+                .withClientConfiguration(config)
+                .withCredentials(credentialsProvider)
+                .withEndpointConfiguration(endpointConfiguration)
+                .withRegion(region.getName())
+                .build();
     }
 
-    /**
-     * Create client using AWSCredentials
-     *
-     * @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
-     */
-    @Deprecated
-    @Override
-    protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
-        getLogger().debug("Creating client with aws credentials");
-        return new AmazonDynamoDBClient(credentials, config);
-    }
 
     protected Object getValue(final ProcessContext context, final PropertyDescriptor type, final PropertyDescriptor value, final Map<String, String> attributes) {
         if ( context.getProperty(type).getValue().equals(ALLOWABLE_VALUE_STRING.getValue())) {
@@ -348,6 +343,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
         }
     }
 
+
     @OnStopped
     public void onStopped() {
         this.dynamoDB = null;
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractWriteDynamoDBProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractWriteDynamoDBProcessor.java
deleted file mode 100644
index fc88037036..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractWriteDynamoDBProcessor.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.aws.dynamodb;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-
-import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
-import com.amazonaws.services.dynamodbv2.model.AttributeValue;
-import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
-import com.amazonaws.services.dynamodbv2.model.WriteRequest;
-
-public abstract class AbstractWriteDynamoDBProcessor extends AbstractDynamoDBProcessor {
-
-    /**
-     * Helper method to handle unprocessed items items
-     * @param session process session
-     * @param keysToFlowFileMap map of flow db primary key to flow file
-     * @param table dynamodb table
-     * @param hashKeyName the hash key name
-     * @param hashKeyValueType the hash key value
-     * @param rangeKeyName the range key name
-     * @param rangeKeyValueType range key value
-     * @param outcome the write outcome
-     */
-    protected void handleUnprocessedItems(final ProcessSession session, Map<ItemKeys, FlowFile> keysToFlowFileMap, final String table, final String hashKeyName, final String hashKeyValueType,
-            final String rangeKeyName, final String rangeKeyValueType, BatchWriteItemOutcome outcome) {
-        BatchWriteItemResult result = outcome.getBatchWriteItemResult();
-
-        // Handle unprocessed items
-        List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
-        if ( unprocessedItems != null && unprocessedItems.size() > 0 ) {
-            for ( WriteRequest request : unprocessedItems) {
-                Map<String,AttributeValue> item = getRequestItem(request);
-                Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
-                Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
-
-                sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
-            }
-        }
-    }
-
-    /**
-     * Get the request item key and attribute value
-     * @param writeRequest write request
-     * @return Map of keys and values
-     */
-    protected abstract Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest);
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/ItemKeys.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/ItemKeys.java
index 9e53457015..1c84db711b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/ItemKeys.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/ItemKeys.java
@@ -16,10 +16,7 @@
  */
 package org.apache.nifi.processors.aws.dynamodb;
 
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
+import java.util.Objects;
 
 /**
  * Utility class to keep a map of keys and flow files
@@ -29,25 +26,35 @@ class ItemKeys {
     protected Object hashKey = "";
     protected Object rangeKey = "";
 
-    public ItemKeys(Object hashKey, Object rangeKey) {
-        if ( hashKey != null )
+    public ItemKeys(final Object hashKey, final Object rangeKey) {
+        if (hashKey != null) {
             this.hashKey = hashKey;
-        if ( rangeKey != null )
+        }
+
+        if (rangeKey != null) {
             this.rangeKey = rangeKey;
+        }
     }
 
     @Override
     public String toString() {
-        return ToStringBuilder.reflectionToString(this,ToStringStyle.SHORT_PREFIX_STYLE);
+        return "ItemKeys[hashKey=" + hashKey + ", rangeKey=" + rangeKey + "]";
     }
 
     @Override
-    public int hashCode() {
-        return HashCodeBuilder.reflectionHashCode(this, false);
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        final ItemKeys itemKeys = (ItemKeys) o;
+        return Objects.equals(hashKey, itemKeys.hashKey) && Objects.equals(rangeKey, itemKeys.rangeKey);
     }
 
     @Override
-    public boolean equals(Object other) {
-        return EqualsBuilder.reflectionEquals(this, other, false);
+    public int hashCode() {
+        return Objects.hash(hashKey, rangeKey);
     }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisProcessorUtils.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisProcessorUtils.java
index adae7783a0..84b966cbb1 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisProcessorUtils.java
@@ -49,22 +49,24 @@ public class KinesisProcessorUtils {
      */
     public static List<FlowFile> filterMessagesByMaxSize(final ProcessSession session, final int batchSize, final long maxBufferSizeBytes,
                                                          final String errorMessageAttribute, final ComponentLog logger) {
-        final List<FlowFile> flowFiles = new ArrayList<>(batchSize);
+        final List<FlowFile> flowFiles = new ArrayList<>();
 
         long currentBufferSizeBytes = 0;
         for (int i = 0; (i < batchSize) && (currentBufferSizeBytes <= maxBufferSizeBytes); i++) {
             final FlowFile flowFileCandidate = session.get();
-            if (flowFileCandidate != null) {
-                if (flowFileCandidate.getSize() > MAX_MESSAGE_SIZE) {
-                    handleFlowFileTooBig(session, flowFileCandidate, errorMessageAttribute, logger);
-                    continue;
-                }
-
-                currentBufferSizeBytes += flowFileCandidate.getSize();
+            if (flowFileCandidate == null) {
+                break;
+            }
 
-                flowFiles.add(flowFileCandidate);
+            if (flowFileCandidate.getSize() > MAX_MESSAGE_SIZE) {
+                handleFlowFileTooBig(session, flowFileCandidate, errorMessageAttribute, logger);
+                continue;
             }
+
+            currentBufferSizeBytes += flowFileCandidate.getSize();
+            flowFiles.add(flowFileCandidate);
         }
+
         return flowFiles;
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
deleted file mode 100644
index 15d12462aa..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/AbstractKinesisFirehoseProcessor.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.aws.kinesis.firehose;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
-import software.amazon.awssdk.services.firehose.FirehoseClient;
-import software.amazon.awssdk.services.firehose.FirehoseClientBuilder;
-
-/**
- * This class is the base class for Kinesis Firehose processors
- */
-public abstract class AbstractKinesisFirehoseProcessor extends AbstractAwsSyncProcessor<FirehoseClient, FirehoseClientBuilder> {
-
-    public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
-            .name("Amazon Kinesis Firehose Delivery Stream Name")
-            .description("The name of kinesis firehose delivery stream")
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
-            .name("Batch Size")
-            .description("Batch size for messages (1-500).")
-            .defaultValue("250")
-            .required(false)
-            .addValidator(StandardValidators.createLongValidator(1, 500, true))
-            .sensitive(false)
-            .build();
-
-    public static final PropertyDescriptor MAX_MESSAGE_BUFFER_SIZE_MB = new PropertyDescriptor.Builder()
-            .name("Max message buffer size")
-            .description("Max message buffer")
-            .defaultValue("1 MB")
-            .required(false)
-            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .sensitive(false)
-            .build();
-
-    @Override
-    protected FirehoseClientBuilder createClientBuilder(final ProcessContext context) {
-        return FirehoseClient.builder();
-    }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamAsyncProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamAsyncProcessor.java
deleted file mode 100644
index 2622b6cf49..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamAsyncProcessor.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.aws.kinesis.stream;
-
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processors.aws.v2.AbstractAwsAsyncProcessor;
-import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
-import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder;
-
-/**
- * This class is the base class for kinesis stream processors that use the async KinesisClient
- */
-public abstract class AbstractKinesisStreamAsyncProcessor extends AbstractAwsAsyncProcessor<KinesisAsyncClient, KinesisAsyncClientBuilder>
-implements KinesisStreamProcessor {
-
-    @Override
-    protected KinesisAsyncClientBuilder createClientBuilder(final ProcessContext context) {
-        return KinesisAsyncClient.builder();
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamSyncProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamSyncProcessor.java
deleted file mode 100644
index 5c03d43a9c..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/AbstractKinesisStreamSyncProcessor.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.aws.kinesis.stream;
-
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
-import software.amazon.awssdk.services.kinesis.KinesisClient;
-import software.amazon.awssdk.services.kinesis.KinesisClientBuilder;
-
-/**
- * This class is the base class for kinesis stream processors that use the synchronized KinesisClient
- */
-public abstract class AbstractKinesisStreamSyncProcessor extends AbstractAwsSyncProcessor<KinesisClient, KinesisClientBuilder>
-implements KinesisStreamProcessor {
-
-    @Override
-    protected KinesisClientBuilder createClientBuilder(final ProcessContext context) {
-        return KinesisClient.builder();
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/KinesisStreamProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/KinesisStreamProcessor.java
deleted file mode 100644
index 73040e376c..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/KinesisStreamProcessor.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.aws.kinesis.stream;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.util.StandardValidators;
-
-public interface KinesisStreamProcessor {
-
-    PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
-            .name("kinesis-stream-name")
-            .displayName("Amazon Kinesis Stream Name")
-            .description("The name of Kinesis Stream")
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
deleted file mode 100644
index 9b0050ebe2..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/lambda/AbstractAWSLambdaProcessor.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.aws.lambda;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
-
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.lambda.AWSLambdaClient;
-
-/**
- * This class is the base class for invoking aws lambda function
- */
-public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsProviderProcessor<AWSLambdaClient> {
-
-    public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
-            .name("Amazon Lambda Name")
-            .description("The Lambda Function Name")
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_QUALIFIER = new PropertyDescriptor.Builder()
-            .name("Amazon Lambda Qualifier (version)")
-            .description("The Lambda Function Version")
-            .defaultValue("$LATEST")
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    /**
-     * Create client using aws credentials provider. This is the preferred way for creating clients
-     */
-    @Override
-    protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
-        getLogger().info("Creating client using aws credentials provider");
-
-        return new AWSLambdaClient(credentialsProvider, config);
-    }
-
-    /**
-     * Create client using AWSCredentials
-     *
-     * @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
-     */
-    @Override
-    protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
-        getLogger().info("Creating client using aws credentials");
-
-        return new AWSLambdaClient(credentials, config);
-    }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
index f236f67ef8..9d8f643c44 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java
@@ -18,13 +18,15 @@ package org.apache.nifi.processors.aws.s3;
 
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.Signer;
+import com.amazonaws.client.builder.AwsClientBuilder;
 import com.amazonaws.regions.Region;
 import com.amazonaws.regions.Regions;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Builder;
 import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
 import com.amazonaws.services.s3.model.AccessControlList;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
@@ -40,6 +42,8 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.ConfigVerificationResult.Outcome;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
@@ -48,9 +52,6 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
-import org.apache.nifi.processors.aws.AwsClientDetails;
-import org.apache.nifi.processors.aws.AwsPropertyDescriptors;
 import org.apache.nifi.processors.aws.signer.AwsCustomSignerUtil;
 import org.apache.nifi.processors.aws.signer.AwsSignerType;
 
@@ -59,6 +60,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Consumer;
 
 import static java.lang.String.format;
 import static org.apache.nifi.processors.aws.signer.AwsSignerType.AWS_S3_V2_SIGNER;
@@ -126,13 +128,17 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .defaultValue("${s3.owner}")
             .build();
-    public static final PropertyDescriptor BUCKET = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor BUCKET_WITHOUT_DEFAULT_VALUE = new PropertyDescriptor.Builder()
             .name("Bucket")
             .description("The S3 Bucket to interact with")
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .required(true)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
+    public static final PropertyDescriptor BUCKET_WITH_DEFAULT_VALUE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(BUCKET_WITHOUT_DEFAULT_VALUE)
+            .defaultValue("${s3.bucket}")
+            .build();
     public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
             .name("Object Key")
             .description("The S3 Object Key to use. This is analogous to a filename for traditional file systems.")
@@ -155,13 +161,25 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
             .build();
 
     public static final PropertyDescriptor S3_CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_CLASS_NAME)
+            .name("custom-signer-class-name")
+            .displayName("Custom Signer Class Name")
+            .description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
             .dependsOn(SIGNER_OVERRIDE, CUSTOM_SIGNER)
             .build();
 
     public static final PropertyDescriptor S3_CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_MODULE_LOCATION)
+            .name("custom-signer-module-location")
+            .displayName("Custom Signer Module Location")
+            .description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
             .dependsOn(SIGNER_OVERRIDE, CUSTOM_SIGNER)
+            .dynamicallyModifiesClasspath(true)
             .build();
 
     public static final String S3_REGION_ATTRIBUTE = "s3.region" ;
@@ -170,7 +188,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
             "Uses '" + S3_REGION_ATTRIBUTE + "' FlowFile attribute as region.");
 
     public static final PropertyDescriptor S3_REGION = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AbstractAWSProcessor.REGION)
+            .fromPropertyDescriptor(REGION)
             .allowableValues(getAvailableS3Regions())
             .build();
 
@@ -203,36 +221,46 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
      * Create client using credentials provider. This is the preferred way for creating clients
      */
     @Override
-    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region,
+                                          final ClientConfiguration config, final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         getLogger().info("Creating client with credentials provider");
         initializeSignerOverride(context, config);
         AmazonS3EncryptionService encryptionService = context.getProperty(ENCRYPTION_SERVICE).asControllerService(AmazonS3EncryptionService.class);
-        AmazonS3Client s3 = null;
 
+        final Consumer<AmazonS3Builder<?, ?>> clientBuilder = builder -> {
+            if (endpointConfiguration == null) {
+                builder.withRegion(region.getName());
+            } else {
+                builder.withEndpointConfiguration(endpointConfiguration);
+            }
+            builder.withClientConfiguration(config);
+            builder.withCredentials(credentialsProvider);
+
+            final Boolean useChunkedEncoding = context.getProperty(USE_CHUNKED_ENCODING).asBoolean();
+            if (useChunkedEncoding == Boolean.FALSE) {
+                builder.disableChunkedEncoding();
+            }
+
+            final Boolean usePathStyleAccess = context.getProperty(USE_PATH_STYLE_ACCESS).asBoolean();
+            final boolean endpointOverrideSet = !StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()).isEmpty();
+            if (usePathStyleAccess == Boolean.TRUE || endpointOverrideSet) {
+                builder.withPathStyleAccessEnabled(true);
+            }
+        };
+
+        AmazonS3 s3Client = null;
         if (encryptionService != null) {
-            s3 = encryptionService.createEncryptionClient(credentialsProvider, config);
+            s3Client = encryptionService.createEncryptionClient(clientBuilder);
         }
-
-        if (s3 == null) {
-            s3 = new AmazonS3Client(credentialsProvider, config);
+        if (s3Client == null) {
+            final AmazonS3ClientBuilder builder = AmazonS3Client.builder();
+            clientBuilder.accept(builder);
+            s3Client = builder.build();
         }
 
-        configureClientOptions(context, s3);
-
-        return s3;
+        return (AmazonS3Client) s3Client;
     }
 
-    /**
-     * Create client using AWSCredentials
-     *
-     * @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
-     */
-    @Deprecated
-    @Override
-    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
-        getLogger().info("Creating client with AWS credentials");
-        return createClient(context, new AWSStaticCredentialsProvider(credentials), config);
-    }
 
     @Override
     public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
@@ -264,8 +292,8 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
      * @return The created S3 client
      */
     protected AmazonS3Client getS3Client(final ProcessContext context, final Map<String, String> attributes) {
-        final AwsClientDetails clientDetails = getAwsClientDetails(context, attributes);
-        return getClient(context, clientDetails);
+        final Region region = resolveRegion(context, attributes);
+        return getClient(context, region);
     }
 
     /**
@@ -275,8 +303,8 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
      * @return The newly created S3 client
      */
     protected AmazonS3Client createClient(final ProcessContext context, final Map<String, String> attributes) {
-        final AwsClientDetails clientDetails = getAwsClientDetails(context, attributes);
-        return createClient(context, clientDetails);
+        final Region region = resolveRegion(context, attributes);
+        return createClient(context, region);
     }
 
     @Override
@@ -287,28 +315,6 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
         }
     }
 
-    private void configureClientOptions(final ProcessContext context, final AmazonS3Client s3) {
-        final S3ClientOptions.Builder builder = S3ClientOptions.builder();
-
-        // disable chunked encoding if "Use Chunked Encoding" has been set to false, otherwise use the default (not disabled)
-        final Boolean useChunkedEncoding = context.getProperty(USE_CHUNKED_ENCODING).asBoolean();
-        if (useChunkedEncoding != null && !useChunkedEncoding) {
-            builder.disableChunkedEncoding();
-        }
-
-        // use PathStyleAccess if "Use Path Style Access" has been set to true, otherwise use the default (false)
-        final Boolean usePathStyleAccess = context.getProperty(USE_PATH_STYLE_ACCESS).asBoolean();
-        if (usePathStyleAccess != null && usePathStyleAccess) {
-            builder.setPathStyleAccess(true);
-        }
-
-        // if ENDPOINT_OVERRIDE is set, use PathStyleAccess
-        if (!StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()).isEmpty()){
-            builder.setPathStyleAccess(true);
-        }
-
-        s3.setS3ClientOptions(builder.build());
-    }
 
     private void initializeSignerOverride(final ProcessContext context, final ClientConfiguration config) {
         final String signer = context.getProperty(SIGNER_OVERRIDE).getValue();
@@ -323,12 +329,6 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
         }
     }
 
-    @Override
-    protected boolean isCustomSignerConfigured(final ProcessContext context) {
-        final String signer = context.getProperty(SIGNER_OVERRIDE).getValue();
-        final AwsSignerType signerType = AwsSignerType.forValue(signer);
-        return signerType == CUSTOM_SIGNER;
-    }
 
     protected Grantee createGrantee(final String value) {
         if (StringUtils.isEmpty(value)) {
@@ -374,9 +374,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
         if (!StringUtils.isEmpty(ownerId)) {
             final Owner owner = new Owner();
             owner.setId(ownerId);
-            if (acl == null) {
-                acl = new AccessControlList();
-            }
+            acl = new AccessControlList();
             acl.setOwner(owner);
         }
 
@@ -423,8 +421,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
         if (e instanceof AmazonS3Exception) {
             flowFile = putAttribute(session, flowFile, "s3.additionalDetails", ((AmazonS3Exception) e).getAdditionalDetails());
         }
-        if (e instanceof AmazonServiceException) {
-            final AmazonServiceException ase = (AmazonServiceException) e;
+        if (e instanceof final AmazonServiceException ase) {
             flowFile = putAttribute(session, flowFile, "s3.statusCode", ase.getStatusCode());
             flowFile = putAttribute(session, flowFile, "s3.errorCode", ase.getErrorCode());
             flowFile = putAttribute(session, flowFile, "s3.errorMessage", ase.getErrorMessage());
@@ -486,8 +483,4 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
         return ArrayUtils.addAll(availableRegions, ATTRIBUTE_DEFINED_REGION);
     }
 
-    private AwsClientDetails getAwsClientDetails(final ProcessContext context, final Map<String, String> attributes) {
-        final Region region = resolveRegion(context, attributes);
-        return new AwsClientDetails(region);
-    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
deleted file mode 100644
index 40857a2ba0..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.aws.sns;
-
-import org.apache.nifi.components.AllowableValue;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
-import software.amazon.awssdk.services.sns.SnsClient;
-import software.amazon.awssdk.services.sns.SnsClientBuilder;
-
-public abstract class AbstractSNSProcessor extends AbstractAwsSyncProcessor<SnsClient, SnsClientBuilder> {
-
-    protected static final AllowableValue ARN_TYPE_TOPIC
-            = new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic");
-    protected static final AllowableValue ARN_TYPE_TARGET
-            = new AllowableValue("Target ARN", "Target ARN", "The ARN is the name of a particular Target, used to notify a specific subscriber");
-
-    public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
-            .name("Amazon Resource Name (ARN)")
-            .description("The name of the resource to which notifications should be published")
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .required(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
-            .name("ARN Type")
-            .description("The type of Amazon Resource Name that is being used.")
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .required(true)
-            .allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
-            .defaultValue(ARN_TYPE_TOPIC.getValue())
-            .build();
-
-
-    @Override
-    protected SnsClientBuilder createClientBuilder(final ProcessContext context) {
-        return SnsClient.builder();
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
deleted file mode 100644
index 817babb30e..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.aws.sqs;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
-import software.amazon.awssdk.services.sqs.SqsClient;
-import software.amazon.awssdk.services.sqs.SqsClientBuilder;
-
-public abstract class AbstractSQSProcessor extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
-
-    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
-            .name("Batch Size")
-            .description("The maximum number of messages to send in a single network request")
-            .required(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .defaultValue("25")
-            .build();
-
-    public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
-            .name("Queue URL")
-            .description("The URL of the queue to act upon")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .required(true)
-            .build();
-
-    @Override
-    protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
-        return SqsClient.builder();
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsAsyncProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsAsyncProcessor.java
index f7475dbbba..659b2a674b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsAsyncProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsAsyncProcessor.java
@@ -24,6 +24,7 @@ import software.amazon.awssdk.core.SdkClient;
 import software.amazon.awssdk.http.TlsKeyManagersProvider;
 import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
 import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.regions.Region;
 
 import javax.net.ssl.TrustManager;
 import java.time.Duration;
@@ -54,10 +55,9 @@ public abstract class AbstractAwsAsyncProcessor<
      * @param context The process context
      * @return The created client
      */
-    @Override
-    public T createClient(final ProcessContext context) {
+    public T createClient(final ProcessContext context, final Region region) {
         final U clientBuilder = createClientBuilder(context);
-        this.configureClientBuilder(clientBuilder, context);
+        this.configureClientBuilder(clientBuilder, region, context);
         return clientBuilder.build();
     }
 
@@ -101,4 +101,5 @@ public abstract class AbstractAwsAsyncProcessor<
 
         return builder.build();
     }
+
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsProcessor.java
index 12c0885a85..5242fd6990 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsProcessor.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.processors.aws.v2;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -78,8 +80,7 @@ import java.util.concurrent.TimeUnit;
  *
  * @see <a href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.html">AwsCredentialsProvider</a>
  */
-public abstract class AbstractAwsProcessor<T extends SdkClient>
-        extends AbstractSessionFactoryProcessor implements VerifiableProcessor, AwsClientProvider<T> {
+public abstract class AbstractAwsProcessor<T extends SdkClient> extends AbstractSessionFactoryProcessor implements VerifiableProcessor {
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
             .name("success")
@@ -97,7 +98,7 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
 
     public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
 
-    public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY;
+    public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY_ID;
 
     public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
 
@@ -187,7 +188,7 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
 
     public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
 
-    private final AwsClientCache<T> awsClientCache = new AwsClientCache<>();
+    private final Cache<Region, T> clientCache = Caffeine.newBuilder().build();
 
     /**
      * Configure the http client on the builder.
@@ -275,14 +276,16 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
 
     @OnStopped
     public void onStopped() {
-        this.awsClientCache.clearCache();
+        clientCache.asMap().values().forEach(SdkClient::close);
+        clientCache.invalidateAll();
+        clientCache.cleanUp();
     }
 
     @Override
     public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
         final List<ConfigVerificationResult> results = new ArrayList<>();
 
-        try (final T client = createClient(context)) {
+        try (final T ignored = createClient(context)) {
             results.add(new ConfigVerificationResult.Builder()
                     .outcome(Outcome.SUCCESSFUL)
                     .verificationStepName("Create Client")
@@ -302,31 +305,27 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
 
     /**
      * Creates an AWS service client from the context or returns an existing client from the cache
-     * @param context The process context
-     * @param  awsClientDetails details of the AWS client
-     * @return The created client
+     * @return The created or cached client
      */
-    protected T getClient(final ProcessContext context, final AwsClientDetails awsClientDetails) {
-        return this.awsClientCache.getOrCreateClient(context, awsClientDetails, this);
+    protected T getClient(final ProcessContext context, final Region region) {
+        return clientCache.get(region, ignored -> createClient(context, region));
     }
 
     protected T getClient(final ProcessContext context) {
-        final AwsClientDetails awsClientDetails = new AwsClientDetails(getRegion(context));
-        return getClient(context, awsClientDetails);
+        return getClient(context, getRegion(context));
     }
 
     protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
-    void configureClientBuilder(final B clientBuilder, final ProcessContext context) {
-        configureClientBuilder(clientBuilder, context, ENDPOINT_OVERRIDE);
+    void configureClientBuilder(final B clientBuilder, final Region region, final ProcessContext context) {
+        configureClientBuilder(clientBuilder, region, context, ENDPOINT_OVERRIDE);
     }
 
     protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
-    void configureClientBuilder(final B clientBuilder, final ProcessContext context, final PropertyDescriptor endpointOverrideDescriptor) {
+    void configureClientBuilder(final B clientBuilder, final Region region, final ProcessContext context, final PropertyDescriptor endpointOverrideDescriptor) {
         clientBuilder.overrideConfiguration(builder -> builder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, DEFAULT_USER_AGENT));
         clientBuilder.overrideConfiguration(builder -> builder.retryPolicy(RetryPolicy.none()));
         this.configureHttpClient(clientBuilder, context);
 
-        final Region region = getRegion(context);
         if (region != null) {
             clientBuilder.region(region);
         }
@@ -391,10 +390,10 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
         return region;
     }
 
-    protected  void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder, final PropertyDescriptor endpointOverrideDescriptor) {
+    protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder<?, ?> clientBuilder, final PropertyDescriptor endpointOverrideDescriptor) {
         // if the endpoint override has been configured, set the endpoint.
         // (per Amazon docs this should only be configured at client creation)
-        if (getSupportedPropertyDescriptors().contains(endpointOverrideDescriptor)) {
+        if (endpointOverrideDescriptor != null && getSupportedPropertyDescriptors().contains(endpointOverrideDescriptor)) {
             final String endpointOverride = StringUtils.trimToEmpty(context.getProperty(endpointOverrideDescriptor).evaluateAttributeExpressions().getValue());
 
             if (!endpointOverride.isEmpty()) {
@@ -405,9 +404,6 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
         }
     }
 
-    protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder) {
-        configureEndpoint(context, clientBuilder, ENDPOINT_OVERRIDE);
-    }
 
     /**
      * Get credentials provider using the {@link AwsCredentialsProvider}
@@ -438,4 +434,19 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
 
         return AnonymousCredentialsProvider.create();
     }
+
+
+    protected T createClient(final ProcessContext context) {
+        return createClient(context, getRegion(context));
+    }
+
+    /**
+     * Creates an AWS client using process context and AWS client details.
+     *
+     * @param context process context
+     * @param region  the AWS Region
+     * @return AWS client
+     */
+    protected abstract T createClient(final ProcessContext context, final Region region);
+
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsSyncProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsSyncProcessor.java
index 3c7f6447c4..62df38dcaf 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsSyncProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AbstractAwsSyncProcessor.java
@@ -24,6 +24,7 @@ import software.amazon.awssdk.core.SdkClient;
 import software.amazon.awssdk.http.SdkHttpClient;
 import software.amazon.awssdk.http.TlsKeyManagersProvider;
 import software.amazon.awssdk.http.apache.ApacheHttpClient;
+import software.amazon.awssdk.regions.Region;
 
 import javax.net.ssl.TrustManager;
 import java.net.URI;
@@ -55,10 +56,13 @@ public abstract class AbstractAwsSyncProcessor<
      * @param context The process context
      * @return The created client
      */
-    @Override
     public T createClient(final ProcessContext context) {
+        return createClient(context, getRegion(context));
+    }
+
+    public T createClient(final ProcessContext context, final Region region) {
         final U clientBuilder = createClientBuilder(context);
-        this.configureClientBuilder(clientBuilder, context);
+        this.configureClientBuilder(clientBuilder, region, context);
         return clientBuilder.build();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientCache.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientCache.java
deleted file mode 100644
index bf836699b2..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientCache.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.aws.v2;
-
-import com.github.benmanes.caffeine.cache.Cache;
-import com.github.benmanes.caffeine.cache.Caffeine;
-import org.apache.nifi.processor.ProcessContext;
-import software.amazon.awssdk.core.SdkClient;
-
-public class AwsClientCache<T extends SdkClient> {
-
-    private final Cache<AwsClientDetails, T> clientCache = Caffeine.newBuilder().build();
-
-    public T getOrCreateClient(final ProcessContext context, final AwsClientDetails clientDetails, final AwsClientProvider<T> provider) {
-        return clientCache.get(clientDetails, ignored -> provider.createClient(context));
-    }
-
-    public void closeClients() {
-        clientCache.asMap().values().stream().forEach(SdkClient::close);
-    }
-
-    public void clearCache() {
-        closeClients();
-        clientCache.invalidateAll();
-        clientCache.cleanUp();
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientDetails.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientDetails.java
deleted file mode 100644
index b8148d6093..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientDetails.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.aws.v2;
-
-import software.amazon.awssdk.regions.Region;
-
-import java.util.Objects;
-
-/**
- * This class contains the AWS client details used to distinguish between the various AWS clients stored in the cache.
- * The class acts as a cache key for @link AwsClientCache.
- * AwsClientDetails contains the region only, since actually the region value may come from the FlowFile attributes.
- */
-public class AwsClientDetails {
-    private Region region;
-
-    public AwsClientDetails(Region region) {
-       this.region =  region;
-    }
-
-    public Region getRegion() {
-        return region;
-    }
-
-    public void setRegion(final Region region) {
-        this.region = region;
-    }
-
-    @Override
-    public boolean equals(final Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        final AwsClientDetails that = (AwsClientDetails) o;
-        return Objects.equals(region, that.region);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(region);
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientProvider.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientProvider.java
deleted file mode 100644
index bea56727f8..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/v2/AwsClientProvider.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.aws.v2;
-
-import org.apache.nifi.processor.ProcessContext;
-import software.amazon.awssdk.core.SdkClient;
-
-public interface AwsClientProvider<T extends SdkClient> {
-
-    /**
-     * Creates an AWS client using process context and AWS client details.
-     *
-     * @param context process context
-     * @return AWS client
-     */
-    T createClient(final ProcessContext context);
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
index 1716e4ae29..96daa83b01 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/AbstractAWSGatewayApiProcessor.java
@@ -16,33 +16,12 @@
  */
 package org.apache.nifi.processors.aws.wag;
 
-import static org.apache.commons.lang3.StringUtils.trimToEmpty;
-
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
 import com.amazonaws.http.AmazonHttpClient;
 import com.amazonaws.http.HttpMethodName;
 import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.URLEncoder;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 import org.apache.commons.io.Charsets;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.http.NameValuePair;
@@ -50,7 +29,7 @@ import org.apache.http.client.utils.URLEncodedUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -60,18 +39,37 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayClient;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayClientBuilder;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequest;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequestBuilder;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayResponse;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.lang3.StringUtils.trimToEmpty;
+
 /**
  * This class is the base class for invoking aws gateway api endpoints
  */
-public abstract class AbstractAWSGatewayApiProcessor extends
-                                                     AbstractAWSCredentialsProviderProcessor<GenericApiGatewayClient> {
+public abstract class AbstractAWSGatewayApiProcessor extends AbstractAWSCredentialsProviderProcessor<GenericApiGatewayClient> {
 
     private volatile Set<String> dynamicPropertyNames = new HashSet<>();
     private volatile Pattern regexAttributesToSend = null;
@@ -105,21 +103,23 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     // processing, including when converting http headers, copying attributes, etc.
     // This set includes our strings defined above as well as some standard flowfile
     // attributes.
-    public static final Set<String> IGNORED_ATTRIBUTES = Collections.unmodifiableSet(new HashSet<>(
-        Arrays.asList(STATUS_CODE, STATUS_MESSAGE, RESOURCE_NAME_ATTR, TRANSACTION_ID, "uuid",
-                      "filename", "path")));
+    public static final Set<String> IGNORED_ATTRIBUTES = Set.of(STATUS_CODE,
+            STATUS_MESSAGE,
+            RESOURCE_NAME_ATTR,
+            TRANSACTION_ID,
+            CoreAttributes.UUID.key(),
+            CoreAttributes.FILENAME.key(),
+            CoreAttributes.PATH.key());
 
     public static final PropertyDescriptor PROP_METHOD = new PropertyDescriptor.Builder()
             .name("aws-gateway-http-method")
             .displayName("HTTP Method")
-            .description(
-                "HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS)."
+            .description("HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS)."
                     + "Methods other than POST, PUT and PATCH will be sent without a message body.")
             .required(true)
             .defaultValue("GET")
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .addValidator(StandardValidators
-                    .createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor PROP_AWS_API_KEY = new PropertyDescriptor.Builder()
@@ -140,16 +140,6 @@ public abstract class AbstractAWSGatewayApiProcessor extends
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
-    // we use our own region, because the way the base sets the region after the client is created
-    // resets the endpoint and breaks everything
-    public static final PropertyDescriptor PROP_AWS_GATEWAY_API_REGION = new PropertyDescriptor.Builder()
-            .name("aws-gateway-region")
-            .displayName("Amazon Region")
-            .required(true)
-            .allowableValues(AbstractAWSProcessor.getAvailableRegions())
-            .defaultValue(AbstractAWSProcessor.createAllowableValue(Regions.DEFAULT_REGION).getValue())
-            .build();
-
     public static final PropertyDescriptor PROP_RESOURCE_NAME = new PropertyDescriptor.Builder()
             .name("aws-gateway-resource")
             .displayName("Amazon Gateway Api ResourceName")
@@ -160,13 +150,11 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     public static final PropertyDescriptor PROP_QUERY_PARAMS = new PropertyDescriptor.Builder()
             .name("aws-gateway-query-parameters")
             .displayName("Query Parameters")
-            .description(
-                "The query parameters for this request in the form of Name=Value separated by &")
+            .description("The query parameters for this request in the form of Name=Value separated by &")
             .displayName("Query Parameters")
             .required(false)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .addValidator(StandardValidators
-                    .createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
     public static final PropertyDescriptor PROP_ATTRIBUTES_TO_SEND = new PropertyDescriptor.Builder()
@@ -184,19 +172,16 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     public static final PropertyDescriptor PROP_PUT_OUTPUT_IN_ATTRIBUTE = new PropertyDescriptor.Builder()
             .name("aws-gateway-put-response-body-in-attribute")
             .displayName("Put Response Body In Attribute")
-            .description(
-                "If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
+            .description("If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
                     + "FlowFile. The attribute key to put to is determined by evaluating value of this property. ")
-            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
-                AttributeExpression.ResultType.STRING))
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
     public static final PropertyDescriptor PROP_OUTPUT_RESPONSE_REGARDLESS = new PropertyDescriptor.Builder()
             .name("aws-gateway-always-output-response")
             .displayName("Always Output Response")
-            .description(
-                "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is "
+            .description("Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is "
                     + "or if the processor is configured to put the server response body in the request attribute. In the later configuration a request FlowFile with the "
                     + "response body in the attribute and a typical response FlowFile will be emitted to their respective relationships.")
             .required(false)
@@ -207,8 +192,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     public static final PropertyDescriptor PROP_PENALIZE_NO_RETRY = new PropertyDescriptor.Builder()
             .name("aws-gateway-penalize-no-retry")
             .displayName("Penalize on \"No Retry\"")
-            .description("Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" " +
-                    "relationship.")
+            .description("Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.")
             .required(false)
             .defaultValue("false")
             .allowableValues("true", "false")
@@ -235,10 +219,9 @@ public abstract class AbstractAWSGatewayApiProcessor extends
                     + "In the case of an empty value after evaluating an expression language expression, Content-Type defaults to "
                     + DEFAULT_CONTENT_TYPE)
             .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
-            .addValidator(StandardValidators
-                    .createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+            .addValidator(Validator.VALID)
             .build();
 
     public static final PropertyDescriptor PROP_SEND_BODY = new PropertyDescriptor.Builder()
@@ -262,32 +245,13 @@ public abstract class AbstractAWSGatewayApiProcessor extends
             .allowableValues("true", "false")
             .build();
 
-    public static final PropertyDescriptor PROP_CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
-            .name("aws-gateway-connection-timeout")
-            .displayName("Connection Timeout")
-            .description("Max wait time for connection to remote service.")
-            .required(false)
-            .defaultValue("10 secs")
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROP_READ_TIMEOUT = new PropertyDescriptor.Builder()
-            .name("aws-gateway-read-timeout")
-            .displayName("Read Timeout")
-            .description("Max wait time for response from remote service.")
-            .required(false)
-            .defaultValue("50 secs")
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .build();
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .required(false)
                 .name(propertyDescriptorName)
-                .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
-                                                                     AttributeExpression.ResultType.STRING,
-                                                                     true))
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
                 .dynamic(true)
                 .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES).build();
     }
@@ -335,22 +299,28 @@ public abstract class AbstractAWSGatewayApiProcessor extends
                                                                    .getValue();
                     // user is not expected to encode, that will be done by the aws client
                     // but we may need to when validating
-                    final String encodedInput = URLEncoder.encode(evaluatedInput, "UTF-8");
+                    final String encodedInput = URLEncoder.encode(evaluatedInput, StandardCharsets.UTF_8);
                     final String url = String.format("http://www.foo.com?%s", encodedInput);
                     URI.create(url).toURL();
-                    results.add(new ValidationResult.Builder().subject(PROP_QUERY_PARAMS.getName())
-                                                              .input(input)
-                                                              .explanation("Valid URL params")
-                                                              .valid(true).build());
+
+                    results.add(new ValidationResult.Builder()
+                            .subject(PROP_QUERY_PARAMS.getName())
+                            .input(input)
+                            .explanation("Valid URL params")
+                            .valid(true)
+                            .build());
                 } catch (final Exception e) {
-                    results.add(new ValidationResult.Builder().subject(PROP_QUERY_PARAMS.getName())
-                                                              .input(input).explanation(
-                            "Not a valid set of URL params").valid(false).build());
+                    results.add(new ValidationResult.Builder()
+                            .subject(PROP_QUERY_PARAMS.getName())
+                            .input(input)
+                            .explanation("Not a valid set of URL params")
+                            .valid(false)
+                            .build());
                 }
             }
         }
-        final String method = trimToEmpty(validationContext.getProperty(PROP_METHOD).getValue())
-            .toUpperCase();
+
+        final String method = trimToEmpty(validationContext.getProperty(PROP_METHOD).getValue()).toUpperCase();
 
         // if there are expressions do not validate
         if (!(validationContext.isExpressionLanguageSupported(PROP_METHOD.getName())
@@ -368,30 +338,19 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     }
 
     @Override
-    protected GenericApiGatewayClient createClient(final ProcessContext context,
-                                                   final AWSCredentialsProvider awsCredentialsProvider,
-                                                   final ClientConfiguration clientConfiguration) {
-
+    protected GenericApiGatewayClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                   final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         GenericApiGatewayClientBuilder builder = new GenericApiGatewayClientBuilder()
-            .withCredentials(awsCredentialsProvider).withClientConfiguration(clientConfiguration)
-            .withEndpoint(context.getProperty(PROP_AWS_GATEWAY_API_ENDPOINT).getValue()).withRegion(
-                Region.getRegion(
-                    Regions.fromName(context.getProperty(PROP_AWS_GATEWAY_API_REGION).getValue())));
-        if (context.getProperty(PROP_AWS_API_KEY).isSet()) {
-            builder = builder.withApiKey(context.getProperty(PROP_AWS_API_KEY).evaluateAttributeExpressions().getValue());
-        }
+                .withCredentials(credentialsProvider).withClientConfiguration(config)
+                .withEndpoint(context.getProperty(PROP_AWS_GATEWAY_API_ENDPOINT).getValue())
+                .withRegion(region)
+                .withApiKey(context.getProperty(PROP_AWS_API_KEY).evaluateAttributeExpressions().getValue());
+
         if (providedClient != null) {
             builder = builder.withHttpClient(providedClient);
         }
-        return builder.build();
-    }
 
-    @Override
-    @Deprecated
-    protected GenericApiGatewayClient createClient(final ProcessContext context,
-                                                   final AWSCredentials credentials,
-                                                   final ClientConfiguration clientConfiguration) {
-        return createClient(context, new AWSStaticCredentialsProvider(credentials), clientConfiguration);
+        return builder.build();
     }
 
     protected GenericApiGatewayRequest configureRequest(final ProcessContext context,
@@ -499,8 +458,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
             }
         }
 
-        String contentType = context.getProperty(PROP_CONTENT_TYPE)
-                                    .evaluateAttributeExpressions(requestAttributes).getValue();
+        String contentType = context.getProperty(PROP_CONTENT_TYPE).evaluateAttributeExpressions(requestAttributes).getValue();
         final boolean sendBody = context.getProperty(PROP_SEND_BODY).asBoolean();
         contentType = StringUtils.isBlank(contentType) ? DEFAULT_CONTENT_TYPE : contentType;
         if (methodName == HttpMethodName.PUT || methodName == HttpMethodName.POST
@@ -557,21 +515,23 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     protected Map<String, String> convertAttributesFromHeaders(final GenericApiGatewayResponse responseHttp) {
         // create a new hashmap to store the values from the connection
         final  Map<String, String> map = new HashMap<>();
-        responseHttp.getHttpResponse().getHeaders().entrySet().forEach((entry) -> {
-
-            final String key = entry.getKey();
-            final String value = entry.getValue();
-
+        responseHttp.getHttpResponse().getAllHeaders().forEach((key, headers) -> {
             if (key == null) {
                 return;
             }
+
+            final String joined = headers.stream()
+                    .map(String::trim)
+                    .filter(str -> !str.isEmpty())
+                    .collect(Collectors.joining(","));
+
             // we ignore any headers with no actual values (rare)
-            if (StringUtils.isBlank(value)) {
+            if (StringUtils.isBlank(joined)) {
                 return;
             }
 
             // put the csv into the map
-            map.put(key, value);
+            map.put(key, joined);
         });
 
         return map;
@@ -647,17 +607,17 @@ public abstract class AbstractAWSGatewayApiProcessor extends
     protected void logResponse(final ComponentLog logger, final GenericApiGatewayResponse response) {
         try {
             logger.debug("\nResponse from remote service:\n\t{}\n{}",
-                    new Object[]{response.getHttpResponse().getHttpRequest().getURI().toURL().toExternalForm(), getLogString(response.getHttpResponse().getHeaders())});
+                    new Object[]{response.getHttpResponse().getHttpRequest().getURI().toURL().toExternalForm(), getLogString(response.getHttpResponse().getAllHeaders())});
         } catch (MalformedURLException e) {
             logger.debug(e.getMessage());
         }
     }
 
-    protected String getLogString(final Map<String, String> map) {
+    protected String getLogString(final Map<String, ?> map) {
         final StringBuilder sb = new StringBuilder();
         if (map != null && map.size() > 0) {
-            for (Map.Entry<String, String> entry : map.entrySet()) {
-                String value = entry.getValue();
+            for (Map.Entry<String, ?> entry : map.entrySet()) {
+                final Object value = entry.getValue();
                 sb.append("\t");
                 sb.append(entry.getKey());
                 sb.append(": ");
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/client/GenericApiGatewayClient.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/client/GenericApiGatewayClient.java
index 73bac8982b..5dfb253f35 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/client/GenericApiGatewayClient.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/wag/client/GenericApiGatewayClient.java
@@ -2,8 +2,10 @@ package org.apache.nifi.processors.aws.wag.client;
 
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceResponse;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.DefaultRequest;
+import com.amazonaws.Response;
 import com.amazonaws.auth.AWS4Signer;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.http.AmazonHttpClient;
@@ -12,6 +14,7 @@ import com.amazonaws.http.HttpMethodName;
 import com.amazonaws.http.HttpResponseHandler;
 import com.amazonaws.http.JsonResponseHandler;
 import com.amazonaws.internal.auth.DefaultSignerProvider;
+import com.amazonaws.protocol.json.JsonErrorResponseMetadata;
 import com.amazonaws.protocol.json.JsonOperationMetadata;
 import com.amazonaws.protocol.json.SdkStructuredPlainJsonFactory;
 import com.amazonaws.regions.Region;
@@ -19,9 +22,9 @@ import com.amazonaws.transform.JsonErrorUnmarshaller;
 import com.amazonaws.transform.JsonUnmarshallerContext;
 import com.amazonaws.transform.Unmarshaller;
 import com.fasterxml.jackson.databind.JsonNode;
+
 import java.io.InputStream;
 import java.net.URI;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -35,12 +38,14 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
     private final AWSCredentialsProvider credentials;
     private final String apiKey;
     private final AWS4Signer signer;
+    private final URI endpoint;
+    private final String region;
 
     GenericApiGatewayClient(ClientConfiguration clientConfiguration, String endpoint, Region region,
                             AWSCredentialsProvider credentials, String apiKey, AmazonHttpClient httpClient) {
         super(clientConfiguration);
-        setRegion(region);
-        setEndpoint(endpoint);
+        this.endpoint = URI.create(endpoint);
+        this.region = region.getName();
         this.credentials = credentials;
         this.apiKey = apiKey;
         this.signer = new AWS4Signer();
@@ -50,14 +55,16 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
         final JsonOperationMetadata metadata = new JsonOperationMetadata().withHasStreamingSuccessResponse(false).withPayloadJson(false);
         final Unmarshaller<GenericApiGatewayResponse, JsonUnmarshallerContext> responseUnmarshaller = in -> new GenericApiGatewayResponse(in.getHttpResponse());
         this.responseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createResponseHandler(metadata, responseUnmarshaller);
-        JsonErrorUnmarshaller defaultErrorUnmarshaller = new JsonErrorUnmarshaller(GenericApiGatewayException.class, null) {
+
+        final JsonErrorResponseMetadata errorResponseMetadata = new JsonErrorResponseMetadata();
+        final JsonErrorUnmarshaller defaultErrorUnmarshaller = new JsonErrorUnmarshaller(GenericApiGatewayException.class, null) {
             @Override
-            public AmazonServiceException unmarshall(JsonNode jsonContent) throws Exception {
+            public AmazonServiceException unmarshall(final JsonNode jsonContent) {
                 return new GenericApiGatewayException(jsonContent.toString());
             }
         };
-        this.errorResponseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createErrorResponseHandler(
-                Collections.singletonList(defaultErrorUnmarshaller), null);
+
+        this.errorResponseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createErrorResponseHandler(errorResponseMetadata, List.of(defaultErrorUnmarshaller));
 
         if (httpClient != null) {
             super.client = httpClient;
@@ -80,18 +87,25 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
         if (parameters != null) {
             request.setParameters(parameters);
         }
-        return this.client.execute(request, responseHandler, errorResponseHandler, executionContext).getAwsResponse();
+
+        final Response<AmazonWebServiceResponse<GenericApiGatewayResponse>> response = client.requestExecutionBuilder()
+                .request(request)
+                .errorResponseHandler(errorResponseHandler)
+                .executionContext(executionContext)
+                .execute(responseHandler);
+
+        return response.getAwsResponse().getResult();
     }
 
     private ExecutionContext buildExecutionContext() {
-        final ExecutionContext executionContext = ExecutionContext.builder().withSignerProvider(
-                new DefaultSignerProvider(this, signer)).build();
+        final ExecutionContext executionContext = ExecutionContext.builder()
+                .withSignerProvider(new DefaultSignerProvider(this, signer))
+                .build();
         executionContext.setCredentialsProvider(credentials);
-        executionContext.setSigner(signer);
         return executionContext;
     }
 
-    private Map<String, String> buildRequestHeaders(Map<String, String> headers, String apiKey) {
+    private Map<String, String> buildRequestHeaders(Map<String, String> headers, final String apiKey) {
         if (headers == null) {
             headers = new HashMap<>();
         }
@@ -105,6 +119,10 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
         return this.endpoint;
     }
 
+    public String getRegion() {
+        return region;
+    }
+
     @Override
     protected String getServiceNameIntern() {
         return API_GATEWAY_SERVICE_NAME;
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-parameter-providers/src/main/java/org/apache/nifi/parameter/aws/AwsSecretsManagerParameterProvider.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-parameter-providers/src/main/java/org/apache/nifi/parameter/aws/AwsSecretsManagerParameterProvider.java
index 7f9e4239f4..7fa9c42fc5 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-parameter-providers/src/main/java/org/apache/nifi/parameter/aws/AwsSecretsManagerParameterProvider.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-parameter-providers/src/main/java/org/apache/nifi/parameter/aws/AwsSecretsManagerParameterProvider.java
@@ -34,6 +34,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.http.conn.ssl.DefaultHostnameVerifier;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
@@ -52,8 +53,6 @@ import org.apache.nifi.ssl.SSLContextService;
 
 import javax.net.ssl.SSLContext;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -119,13 +118,13 @@ public class AwsSecretsManagerParameterProvider extends AbstractParameterProvide
 
     private static final String DEFAULT_USER_AGENT = "NiFi";
     private static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
-    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+    private static final List<PropertyDescriptor> PROPERTIES = List.of(
             SECRET_NAME_PATTERN,
             REGION,
             AWS_CREDENTIALS_PROVIDER_SERVICE,
             TIMEOUT,
             SSL_CONTEXT_SERVICE
-    ));
+    );
 
     private final ObjectMapper objectMapper = new ObjectMapper();
 
@@ -249,7 +248,7 @@ public class AwsSecretsManagerParameterProvider extends AbstractParameterProvide
         final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         if (sslContextService != null) {
             final SSLContext sslContext = sslContextService.createContext();
-            SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, SdkTLSSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER);
+            SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
             config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
         }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
index a41f73c250..2dcd4a70af 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
@@ -141,6 +141,12 @@
             <groupId>com.amazonaws</groupId>
             <artifactId>aws-java-sdk-textract</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>localstack</artifactId>
+            <version>1.19.1</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java
index 142d10c76f..aae4594509 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
 
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.internal.StaticCredentialsProvider;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
@@ -36,22 +36,22 @@ public class AccessKeyPairCredentialsStrategy extends AbstractCredentialsStrateg
 
     public AccessKeyPairCredentialsStrategy() {
         super("Access Key Pair", new PropertyDescriptor[] {
-                CredentialPropertyDescriptors.ACCESS_KEY,
+                CredentialPropertyDescriptors.ACCESS_KEY_ID,
                 CredentialPropertyDescriptors.SECRET_KEY
         });
     }
 
     @Override
     public AWSCredentialsProvider getCredentialsProvider(final PropertyContext propertyContext) {
-        final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY).evaluateAttributeExpressions().getValue();
+        final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID).evaluateAttributeExpressions().getValue();
         final String secretKey = propertyContext.getProperty(CredentialPropertyDescriptors.SECRET_KEY).evaluateAttributeExpressions().getValue();
         final BasicAWSCredentials credentials = new BasicAWSCredentials(accessKey, secretKey);
-        return new StaticCredentialsProvider(credentials);
+        return new AWSStaticCredentialsProvider(credentials);
     }
 
     @Override
     public AwsCredentialsProvider getAwsCredentialsProvider(final PropertyContext propertyContext) {
-        final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY).evaluateAttributeExpressions().getValue();
+        final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID).evaluateAttributeExpressions().getValue();
         final String secretKey = propertyContext.getProperty(CredentialPropertyDescriptors.SECRET_KEY).evaluateAttributeExpressions().getValue();
         return software.amazon.awssdk.auth.credentials.StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey));
     }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java
index fef0ad61cc..8afd3bbc9a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
 
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
 import com.amazonaws.auth.AnonymousAWSCredentials;
-import com.amazonaws.internal.StaticCredentialsProvider;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
 import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
@@ -40,7 +40,7 @@ public class AnonymousCredentialsStrategy extends AbstractBooleanCredentialsStra
     @Override
     public AWSCredentialsProvider getCredentialsProvider(final PropertyContext propertyContext) {
         AnonymousAWSCredentials credentials = new AnonymousAWSCredentials();
-        return new StaticCredentialsProvider(credentials);
+        return new AWSStaticCredentialsProvider(credentials);
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java
index 16a0cebec0..decc4404b4 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java
@@ -19,7 +19,9 @@ package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
 import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -150,7 +152,6 @@ public class AssumeRoleCredentialsStrategy extends AbstractCredentialsStrategy {
         final String assumeRoleSTSSigner = propertyContext.getProperty(ASSUME_ROLE_STS_SIGNER_OVERRIDE).getValue();
         final SSLContextService sslContextService = propertyContext.getProperty(ASSUME_ROLE_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
 
-        STSAssumeRoleSessionCredentialsProvider.Builder builder;
         final ClientConfiguration config = new ClientConfiguration();
 
         if (sslContextService != null) {
@@ -175,26 +176,25 @@ public class AssumeRoleCredentialsStrategy extends AbstractCredentialsStrategy {
             config.withSignerOverride(assumeRoleSTSSigner);
         }
 
-        AWSSecurityTokenServiceClient securityTokenService = new AWSSecurityTokenServiceClient(primaryCredentialsProvider, config);
+        AWSSecurityTokenServiceClientBuilder securityTokenServiceBuilder = AWSSecurityTokenServiceClient.builder()
+                .withCredentials(primaryCredentialsProvider)
+                .withRegion(assumeRoleSTSRegion)
+                .withClientConfiguration(config);
+
         if (assumeRoleSTSEndpoint != null && !assumeRoleSTSEndpoint.isEmpty()) {
-            if (assumeRoleSTSSignerType == CUSTOM_SIGNER) {
-                securityTokenService.setEndpoint(assumeRoleSTSEndpoint, securityTokenService.getServiceName(), assumeRoleSTSRegion);
-            } else {
-                securityTokenService.setEndpoint(assumeRoleSTSEndpoint);
-            }
+            AwsClientBuilder.EndpointConfiguration endpointConfiguration = new AwsClientBuilder.EndpointConfiguration(assumeRoleSTSEndpoint, assumeRoleSTSRegion);
+            securityTokenServiceBuilder.withEndpointConfiguration(endpointConfiguration);
         }
 
-        builder = new STSAssumeRoleSessionCredentialsProvider
-                .Builder(assumeRoleArn, assumeRoleName)
-                .withStsClient(securityTokenService)
+        STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider.Builder(assumeRoleArn, assumeRoleName)
+                .withStsClient(securityTokenServiceBuilder.build())
                 .withRoleSessionDurationSeconds(maxSessionTime);
 
         if (assumeRoleExternalId != null && !assumeRoleExternalId.isEmpty()) {
-            builder = builder.withExternalId(assumeRoleExternalId);
+            builder.withExternalId(assumeRoleExternalId);
         }
 
         final AWSCredentialsProvider credsProvider = builder.build();
-
         return credsProvider;
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerService.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerService.java
index ef06d3d951..00bc0833c6 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerService.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerService.java
@@ -38,7 +38,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ACCESS_KEY;
+import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ACCESS_KEY_ID;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_EXTERNAL_ID;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_HOST;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_PORT;
@@ -83,7 +83,7 @@ public class AWSCredentialsProviderControllerService extends AbstractControllerS
     static {
         final List<PropertyDescriptor> props = new ArrayList<>();
         props.add(USE_DEFAULT_CREDENTIALS);
-        props.add(ACCESS_KEY);
+        props.add(ACCESS_KEY_ID);
         props.add(SECRET_KEY);
         props.add(CREDENTIALS_FILE);
         props.add(PROFILE_NAME);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDB.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDB.java
index 7f0b3270e5..dd92079987 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDB.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDB.java
@@ -16,12 +16,14 @@
  */
 package org.apache.nifi.processors.aws.dynamodb;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
+import com.amazonaws.services.dynamodbv2.model.AttributeValue;
+import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
+import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -38,13 +40,11 @@ import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
-import com.amazonaws.services.dynamodbv2.model.AttributeValue;
-import com.amazonaws.services.dynamodbv2.model.WriteRequest;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 @SupportsBatching
 @SeeAlso({GetDynamoDB.class, PutDynamoDB.class, PutDynamoDBRecord.class})
@@ -69,7 +69,7 @@ import com.amazonaws.services.dynamodbv2.model.WriteRequest;
     @ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_HASH_KEY_VALUE, description = "Items hash key value" ),
     @ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_RANGE_KEY_VALUE, description = "Items range key value" ),
     })
-public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
+public class DeleteDynamoDB extends AbstractDynamoDBProcessor {
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
             Arrays.asList(TABLE, HASH_KEY_NAME, RANGE_KEY_NAME, HASH_KEY_VALUE, RANGE_KEY_VALUE,
@@ -128,10 +128,20 @@ public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
         final DynamoDB dynamoDB = getDynamoDB(context);
 
         try {
-            BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
-
-            handleUnprocessedItems(session, keysToFlowFileMap, table, hashKeyName, hashKeyValueType, rangeKeyName,
-               rangeKeyValueType, outcome);
+            final BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
+            final BatchWriteItemResult result = outcome.getBatchWriteItemResult();
+
+            // Handle unprocessed items
+            final List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
+            if (unprocessedItems != null && unprocessedItems.size() > 0) {
+                for (final WriteRequest request : unprocessedItems) {
+                    final Map<String, AttributeValue> item = request.getDeleteRequest().getKey();
+                    final Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
+                    final Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
+
+                    sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
+                }
+            }
 
             // All non unprocessed items are successful
             for (FlowFile flowFile : keysToFlowFileMap.values()) {
@@ -153,10 +163,4 @@ public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
         }
     }
 
-    /**
-     * {@inheritDoc}
-     */
-    protected Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest) {
-        return writeRequest.getDeleteRequest().getKey();
-    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDB.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDB.java
index aab26ca632..8489437e29 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDB.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDB.java
@@ -16,22 +16,24 @@
  */
 package org.apache.nifi.processors.aws.dynamodb;
 
-import java.io.ByteArrayOutputStream;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
+import com.amazonaws.services.dynamodbv2.model.AttributeValue;
+import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
+import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
 import org.apache.nifi.annotation.behavior.ReadsAttributes;
-import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -42,14 +44,12 @@ import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.document.Item;
-import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
-import com.amazonaws.services.dynamodbv2.model.AttributeValue;
-import com.amazonaws.services.dynamodbv2.model.WriteRequest;
+import java.io.ByteArrayOutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 @SupportsBatching
 @SeeAlso({DeleteDynamoDB.class, GetDynamoDB.class, PutDynamoDBRecord.class})
@@ -78,7 +78,7 @@ import com.amazonaws.services.dynamodbv2.model.WriteRequest;
     @ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_RANGE_KEY_VALUE, description = "Items range key value")
 })
 @SystemResourceConsideration(resource = SystemResource.MEMORY)
-public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
+public class PutDynamoDB extends AbstractDynamoDBProcessor {
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
         Arrays.asList(TABLE, HASH_KEY_NAME, RANGE_KEY_NAME, HASH_KEY_VALUE, RANGE_KEY_VALUE,
@@ -155,10 +155,20 @@ public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
         final DynamoDB dynamoDB = getDynamoDB(context);
 
         try {
-            BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
-
-            handleUnprocessedItems(session, keysToFlowFileMap, table, hashKeyName, hashKeyValueType, rangeKeyName,
-                rangeKeyValueType, outcome);
+            final BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
+            final BatchWriteItemResult result = outcome.getBatchWriteItemResult();
+
+            // Handle unprocessed items
+            final List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
+            if (unprocessedItems != null && unprocessedItems.size() > 0) {
+                for (final WriteRequest request : unprocessedItems) {
+                    final Map<String, AttributeValue> item = request.getPutRequest().getItem();
+                    final Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
+                    final Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
+
+                    sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
+                }
+            }
 
             // Handle any remaining flowfiles
             for (FlowFile flowFile : keysToFlowFileMap.values()) {
@@ -184,11 +194,4 @@ public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
         return (flowFile.getSize() + jsonDocument.length()) < DYNAMODB_MAX_ITEM_SIZE;
     }
 
-    /**
-     * {@inheritDoc}
-     */
-    protected Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest) {
-        return writeRequest.getPutRequest().getItem();
-    }
-
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
index 3043a6e2e7..82d5d0061b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
@@ -25,21 +25,23 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.core.SdkBytes;
 import software.amazon.awssdk.services.firehose.FirehoseClient;
+import software.amazon.awssdk.services.firehose.FirehoseClientBuilder;
 import software.amazon.awssdk.services.firehose.model.PutRecordBatchRequest;
 import software.amazon.awssdk.services.firehose.model.PutRecordBatchResponse;
 import software.amazon.awssdk.services.firehose.model.PutRecordBatchResponseEntry;
 import software.amazon.awssdk.services.firehose.model.Record;
 
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -53,37 +55,67 @@ import java.util.Map;
     @WritesAttribute(attribute = "aws.kinesis.firehose.error.message", description = "Error message on posting message to AWS Kinesis Firehose"),
     @WritesAttribute(attribute = "aws.kinesis.firehose.error.code", description = "Error code for the message when posting to AWS Kinesis Firehose"),
     @WritesAttribute(attribute = "aws.kinesis.firehose.record.id", description = "Record id of the message posted to Kinesis Firehose")})
-public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
+public class PutKinesisFirehose extends AbstractAwsSyncProcessor<FirehoseClient, FirehoseClientBuilder> {
 
-    /**
-     * Kinesis put record response error message
-     */
     public static final String AWS_KINESIS_FIREHOSE_ERROR_MESSAGE = "aws.kinesis.firehose.error.message";
-
-    /**
-     * Kinesis put record response error code
-     */
     public static final String AWS_KINESIS_FIREHOSE_ERROR_CODE = "aws.kinesis.firehose.error.code";
-
-    /**
-     * Kinesis put record response record id
-     */
     public static final String AWS_KINESIS_FIREHOSE_RECORD_ID = "aws.kinesis.firehose.record.id";
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, BATCH_SIZE, MAX_MESSAGE_BUFFER_SIZE_MB, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
-                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
-
-    /**
-     * Max buffer size 1 MB
-     */
-    public static final int MAX_MESSAGE_SIZE = 1000 * 1024;
+    public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
+            .name("Amazon Kinesis Firehose Delivery Stream Name")
+            .description("The name of kinesis firehose delivery stream")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("Batch Size")
+            .description("Batch size for messages (1-500).")
+            .defaultValue("250")
+            .required(false)
+            .addValidator(StandardValidators.createLongValidator(1, 500, true))
+            .sensitive(false)
+            .build();
+
+    public static final PropertyDescriptor MAX_MESSAGE_BUFFER_SIZE_MB = new PropertyDescriptor.Builder()
+            .name("Max message buffer size")
+            .description("Max message buffer")
+            .defaultValue("1 MB")
+            .required(false)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .sensitive(false)
+            .build();
+
+    private static final List<PropertyDescriptor> properties = List.of(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME,
+            BATCH_SIZE,
+            MAX_MESSAGE_BUFFER_SIZE_MB,
+            REGION,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            ENDPOINT_OVERRIDE);
+
+    public static final int MAX_MESSAGE_SIZE = KinesisProcessorUtils.MAX_MESSAGE_SIZE;
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return properties;
     }
 
+    @Override
+    protected FirehoseClientBuilder createClientBuilder(final ProcessContext context) {
+        return FirehoseClient.builder();
+    }
+
+
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
 
@@ -101,27 +133,19 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
             final List<FlowFile> successfulFlowFiles = new ArrayList<>();
 
             // Prepare batch of records
-            for (int i = 0; i < flowFiles.size(); i++) {
-                FlowFile flowFile = flowFiles.get(i);
-
+            for (final FlowFile flowFile : flowFiles) {
                 final String firehoseStreamName = context.getProperty(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();
 
                 session.read(flowFile, in -> recordHash.get(firehoseStreamName).add(Record.builder().data(SdkBytes.fromInputStream(in)).build()));
 
-                if (recordHash.containsKey(firehoseStreamName) == false) {
-                    recordHash.put(firehoseStreamName, new ArrayList<>());
-                }
-
-                if (hashFlowFiles.containsKey(firehoseStreamName) == false) {
-                    hashFlowFiles.put(firehoseStreamName, new ArrayList<>());
-                }
-
-                hashFlowFiles.get(firehoseStreamName).add(flowFile);
+                recordHash.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
+                final List<FlowFile> flowFilesForStream = hashFlowFiles.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
+                flowFilesForStream.add(flowFile);
             }
 
-            for (final Map.Entry<String, List<Record>> entryRecord : recordHash.entrySet()) {
-                final String streamName = entryRecord.getKey();
-                final List<Record> records = entryRecord.getValue();
+            for (final Map.Entry<String, List<Record>> entry : recordHash.entrySet()) {
+                final String streamName = entry.getKey();
+                final List<Record> records = entry.getValue();
 
                 if (records.size() > 0) {
                     // Send the batch
@@ -135,15 +159,15 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
                     final List<PutRecordBatchResponseEntry> responseEntries = response.requestResponses();
                     for (int i = 0; i < responseEntries.size(); i++ ) {
 
-                        final PutRecordBatchResponseEntry entry = responseEntries.get(i);
+                        final PutRecordBatchResponseEntry responseEntry = responseEntries.get(i);
                         FlowFile flowFile = hashFlowFiles.get(streamName).get(i);
 
                         final Map<String,String> attributes = new HashMap<>();
-                        attributes.put(AWS_KINESIS_FIREHOSE_RECORD_ID, entry.recordId());
-                        flowFile = session.putAttribute(flowFile, AWS_KINESIS_FIREHOSE_RECORD_ID, entry.recordId());
-                        if (StringUtils.isNotBlank(entry.errorCode())) {
-                            attributes.put(AWS_KINESIS_FIREHOSE_ERROR_CODE, entry.errorCode());
-                            attributes.put(AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, entry.errorMessage());
+                        attributes.put(AWS_KINESIS_FIREHOSE_RECORD_ID, responseEntry.recordId());
+                        flowFile = session.putAttribute(flowFile, AWS_KINESIS_FIREHOSE_RECORD_ID, responseEntry.recordId());
+                        if (StringUtils.isNotBlank(responseEntry.errorCode())) {
+                            attributes.put(AWS_KINESIS_FIREHOSE_ERROR_CODE, responseEntry.errorCode());
+                            attributes.put(AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, responseEntry.errorMessage());
                             flowFile = session.putAllAttributes(flowFile, attributes);
                             failedFlowFiles.add(flowFile);
                         } else {
@@ -158,15 +182,14 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
 
             if (failedFlowFiles.size() > 0) {
                 session.transfer(failedFlowFiles, REL_FAILURE);
-                getLogger().error("Failed to publish to kinesis firehose {}", new Object[]{failedFlowFiles});
+                getLogger().error("Failed to publish to kinesis firehose {}", failedFlowFiles);
             }
             if (successfulFlowFiles.size() > 0) {
                 session.transfer(successfulFlowFiles, REL_SUCCESS);
-                getLogger().info("Successfully published to kinesis firehose {}", new Object[]{successfulFlowFiles});
+                getLogger().info("Successfully published to kinesis firehose {}", successfulFlowFiles);
             }
-
         } catch (final Exception exception) {
-            getLogger().error("Failed to publish to kinesis firehose {} with exception {}", new Object[]{flowFiles, exception});
+            getLogger().error("Failed to publish to kinesis firehose {} with exception {}", flowFiles, exception);
             session.transfer(flowFiles, REL_FAILURE);
             context.yield();
         }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
index 66d221f822..abd7bd42c4 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
@@ -50,6 +50,7 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
 import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
 import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.processors.aws.v2.AbstractAwsAsyncProcessor;
 import org.apache.nifi.processors.aws.v2.AbstractAwsProcessor;
 import org.apache.nifi.serialization.RecordReaderFactory;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -60,6 +61,7 @@ import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClientBuilder;
 import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
 import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClientBuilder;
 import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder;
 import software.amazon.kinesis.checkpoint.CheckpointConfig;
 import software.amazon.kinesis.common.ConfigsBuilder;
 import software.amazon.kinesis.common.InitialPositionInStream;
@@ -142,7 +144,8 @@ import java.util.stream.Collectors;
 @SystemResourceConsideration(resource = SystemResource.NETWORK, description = "Kinesis Client Library will continually poll for new Records, " +
         "requesting up to a maximum number of Records/bytes per call. This can result in sustained network usage.")
 @SeeAlso(PutKinesisStream.class)
-public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
+public class ConsumeKinesisStream extends AbstractAwsAsyncProcessor<KinesisAsyncClient, KinesisAsyncClientBuilder> {
+
     private static final String CHECKPOINT_CONFIG = "checkpointConfig";
     private static final String COORDINATOR_CONFIG = "coordinatorConfig";
     private static final String LEASE_MANAGEMENT_CONFIG = "leaseManagementConfig";
@@ -166,6 +169,14 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
             InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
     );
 
+    static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
+            .name("kinesis-stream-name")
+            .displayName("Amazon Kinesis Stream Name")
+            .description("The name of Kinesis Stream")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
     public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
             .displayName("Application Name")
             .name("amazon-kinesis-stream-application-name")
@@ -305,21 +316,35 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
                     " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
             .build();
 
-    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
-            Arrays.asList(
-                    // Kinesis Stream specific properties
-                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
-                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
-                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
-                    // generic AWS processor properties
-                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
-            )
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = List.of(
+            // Kinesis Stream specific properties
+            KINESIS_STREAM_NAME,
+            APPLICATION_NAME,
+            RECORD_READER,
+            RECORD_WRITER,
+            REGION,
+            ENDPOINT_OVERRIDE,
+            DYNAMODB_ENDPOINT_OVERRIDE,
+            INITIAL_STREAM_POSITION,
+            STREAM_POSITION_TIMESTAMP,
+            TIMESTAMP_FORMAT,
+            FAILOVER_TIMEOUT,
+            GRACEFUL_SHUTDOWN_TIMEOUT,
+            CHECKPOINT_INTERVAL,
+            NUM_RETRIES,
+            RETRY_WAIT,
+            REPORT_CLOUDWATCH_METRICS,
+
+            // generic AWS processor properties
+            TIMEOUT,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            PROXY_CONFIGURATION_SERVICE
     );
 
-    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<>() {{
-        put("leaseManagementConfig.initialPositionInStream", INITIAL_STREAM_POSITION);
-        put("leaseManagementConfig.failoverTimeMillis", FAILOVER_TIMEOUT);
-    }};
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = Map.of(
+            "leaseManagementConfig.initialPositionInStream", INITIAL_STREAM_POSITION,
+            "leaseManagementConfig.failoverTimeMillis", FAILOVER_TIMEOUT
+    );
 
     private static final Object WORKER_LOCK = new Object();
     private static final String SCHEDULER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Scheduler.class.getSimpleName() + "-";
@@ -459,11 +484,11 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
     private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
                                                                       final String subject, final String input, final String message) {
         return validationResult
-                .explanation(
-                        String.format("Kinesis Client Configuration Builder property with name %s cannot be used with value \"%s\" : %s",
-                                StringUtils.capitalize(subject), input, message)
-                )
-                .valid(false).build();
+                .input(input)
+                .subject(subject)
+                .explanation("Kinesis Client Configuration Builder property with name %s cannot be used with value \"%s\" : %s".formatted(StringUtils.capitalize(subject), input, message))
+                .valid(false)
+                .build();
     }
 
     @Override
@@ -702,13 +727,13 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
 
     private CloudWatchAsyncClient getCloudwatchClient(final ProcessContext context) {
         final CloudWatchAsyncClientBuilder builder = CloudWatchAsyncClient.builder();
-        configureClientBuilder(builder, context, null);
+        configureClientBuilder(builder, getRegion(context), context, null);
         return builder.build();
     }
 
     private DynamoDbAsyncClient getDynamoClient(final ProcessContext context) {
         final DynamoDbAsyncClientBuilder dynamoClientBuilder = DynamoDbAsyncClient.builder();
-        configureClientBuilder(dynamoClientBuilder, context, DYNAMODB_ENDPOINT_OVERRIDE);
+        configureClientBuilder(dynamoClientBuilder, getRegion(context), context, DYNAMODB_ENDPOINT_OVERRIDE);
         return dynamoClientBuilder.build();
     }
 
@@ -807,4 +832,9 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
                 .toInstant().toEpochMilli() // convert to epoch milliseconds for creating Date
         );
     }
+
+    @Override
+    protected KinesisAsyncClientBuilder createClientBuilder(final ProcessContext context) {
+        return KinesisAsyncClient.builder();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
index 3833c097d3..878c81b785 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/PutKinesisStream.java
@@ -33,8 +33,10 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.core.SdkBytes;
 import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.KinesisClientBuilder;
 import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
 import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
 import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse;
@@ -42,8 +44,6 @@ import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry;
 
 import java.io.ByteArrayOutputStream;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -60,7 +60,7 @@ import java.util.Random;
     @WritesAttribute(attribute = "aws.kinesis.sequence.number", description = "Sequence number for the message when posting to AWS Kinesis"),
     @WritesAttribute(attribute = "aws.kinesis.shard.id", description = "Shard id of the message posted to AWS Kinesis")})
 @SeeAlso(ConsumeKinesisStream.class)
-public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
+public class PutKinesisStream extends AbstractAwsSyncProcessor<KinesisClient, KinesisClientBuilder> {
     /**
      * Kinesis put record response error message
      */
@@ -104,14 +104,32 @@ public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
             .sensitive(false)
             .build();
 
-    public static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(AbstractKinesisStreamSyncProcessor.KINESIS_STREAM_NAME)
+    static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
+            .name("kinesis-stream-name")
+            .displayName("Amazon Kinesis Stream Name")
+            .description("The name of Kinesis Stream")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(KINESIS_STREAM_NAME, KINESIS_PARTITION_KEY, BATCH_SIZE, MAX_MESSAGE_BUFFER_SIZE_MB, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
-                AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT, PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
+    public static final List<PropertyDescriptor> properties = List.of(
+            KINESIS_STREAM_NAME,
+            KINESIS_PARTITION_KEY,
+            BATCH_SIZE,
+            MAX_MESSAGE_BUFFER_SIZE_MB,
+            REGION,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            ENDPOINT_OVERRIDE);
 
     /** A random number generator for cases where partition key is not available */
     protected Random randomPartitionKeyGenerator = new Random();
@@ -206,4 +224,10 @@ public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
             context.yield();
         }
     }
+
+    @Override
+    protected KinesisClientBuilder createClientBuilder(final ProcessContext context) {
+        return KinesisClient.builder();
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/lambda/PutLambda.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/lambda/PutLambda.java
index 4ef0bcbcb0..a88ace107e 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/lambda/PutLambda.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/lambda/PutLambda.java
@@ -16,29 +16,11 @@
  */
 package org.apache.nifi.processors.aws.lambda;
 
-import java.io.ByteArrayOutputStream;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang3.StringUtils;
-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.Tags;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-
 import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.lambda.AWSLambdaClient;
 import com.amazonaws.services.lambda.model.InvalidParameterValueException;
 import com.amazonaws.services.lambda.model.InvalidRequestContentException;
@@ -51,6 +33,28 @@ import com.amazonaws.services.lambda.model.ResourceNotFoundException;
 import com.amazonaws.services.lambda.model.TooManyRequestsException;
 import com.amazonaws.services.lambda.model.UnsupportedMediaTypeException;
 import com.amazonaws.util.Base64;
+import org.apache.commons.lang3.StringUtils;
+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.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"amazon", "aws", "lambda", "put"})
@@ -69,66 +73,52 @@ import com.amazonaws.util.Base64;
     @WritesAttribute(attribute = "aws.lambda.exception.status.code", description = "Exception status code on invoking from AWS Lambda"),
     @WritesAttribute(attribute = "aws.lambda.exception.error.type", description = "Exception error type on invoking from AWS Lambda")
     })
-public class PutLambda extends AbstractAWSLambdaProcessor {
-
-    /**
-     * Lambda result function error message
-     */
+public class PutLambda extends AbstractAWSCredentialsProviderProcessor<AWSLambdaClient> {
     public static final String AWS_LAMBDA_RESULT_FUNCTION_ERROR = "aws.lambda.result.function.error";
-
-    /**
-     * Lambda response status code
-     */
     public static final String AWS_LAMBDA_RESULT_STATUS_CODE = "aws.lambda.result.status.code";
-
-    /**
-     * Lambda response log tail (4kb)
-     */
     public static final String AWS_LAMBDA_RESULT_LOG = "aws.lambda.result.log";
-
-    /**
-     * Lambda payload in response
-     */
     public static final String AWS_LAMBDA_RESULT_PAYLOAD = "aws.lambda.result.payload";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_MESSAGE = "aws.lambda.exception.message";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_CAUSE = "aws.lambda.exception.cause";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_ERROR_CODE = "aws.lambda.exception.error.code";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_REQUEST_ID = "aws.lambda.exception.request.id";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_STATUS_CODE = "aws.lambda.exception.status.code";
-
-    /**
-     * Lambda exception field
-     */
     public static final String AWS_LAMBDA_EXCEPTION_ERROR_TYPE = "aws.lambda.exception.error.type";
-
-    /**
-     * Max request body size
-     */
     public static final long MAX_REQUEST_SIZE = 6 * 1000 * 1000;
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(AWS_LAMBDA_FUNCTION_NAME, AWS_LAMBDA_FUNCTION_QUALIFIER, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
-                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
+    static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
+            .name("Amazon Lambda Name")
+            .description("The Lambda Function Name")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor AWS_LAMBDA_FUNCTION_QUALIFIER = new PropertyDescriptor.Builder()
+            .name("Amazon Lambda Qualifier (version)")
+            .description("The Lambda Function Version")
+            .defaultValue("$LATEST")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+
+    public static final List<PropertyDescriptor> properties = List.of(
+            AWS_LAMBDA_FUNCTION_NAME,
+            AWS_LAMBDA_FUNCTION_QUALIFIER,
+            REGION,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            ENDPOINT_OVERRIDE);
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@@ -137,20 +127,17 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
-
         FlowFile flowFile = session.get();
         if (flowFile == null) {
             return;
         }
 
         final String functionName = context.getProperty(AWS_LAMBDA_FUNCTION_NAME).getValue();
-
         final String qualifier = context.getProperty(AWS_LAMBDA_FUNCTION_QUALIFIER).getValue();
 
         // Max size of message is 6 MB
-        if ( flowFile.getSize() > MAX_REQUEST_SIZE) {
-            getLogger().error("Max size for request body is 6mb but was {} for flow file {} for function {}",
-                new Object[]{flowFile.getSize(), flowFile, functionName});
+        if (flowFile.getSize() > MAX_REQUEST_SIZE) {
+            getLogger().error("Max size for request body is 6mb but was {} for flow file {} for function {}", flowFile.getSize(), flowFile, functionName);
             session.transfer(flowFile, REL_FAILURE);
             return;
         }
@@ -161,26 +148,26 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
             final ByteArrayOutputStream baos = new ByteArrayOutputStream();
             session.exportTo(flowFile, baos);
 
-            InvokeRequest invokeRequest = new InvokeRequest()
+            final InvokeRequest invokeRequest = new InvokeRequest()
                 .withFunctionName(functionName)
                 .withLogType(LogType.Tail).withInvocationType(InvocationType.RequestResponse)
                 .withPayload(ByteBuffer.wrap(baos.toByteArray()))
                 .withQualifier(qualifier);
-            long startTime = System.nanoTime();
 
-            InvokeResult result = client.invoke(invokeRequest);
+            final long startTime = System.nanoTime();
+            final InvokeResult result = client.invoke(invokeRequest);
 
             flowFile = session.putAttribute(flowFile, AWS_LAMBDA_RESULT_STATUS_CODE, result.getStatusCode().toString());
 
-            if ( !StringUtils.isBlank(result.getLogResult() )) {
+            if (!StringUtils.isBlank(result.getLogResult())) {
                 flowFile = session.putAttribute(flowFile, AWS_LAMBDA_RESULT_LOG, new String(Base64.decode(result.getLogResult()),Charset.defaultCharset()));
             }
 
-            if ( result.getPayload() != null ) {
+            if (result.getPayload() != null) {
                 flowFile = session.putAttribute(flowFile, AWS_LAMBDA_RESULT_PAYLOAD, new String(result.getPayload().array(),Charset.defaultCharset()));
             }
 
-            if ( ! StringUtils.isBlank(result.getFunctionError()) ){
+            if (!StringUtils.isBlank(result.getFunctionError())) {
                 flowFile = session.putAttribute(flowFile, AWS_LAMBDA_RESULT_FUNCTION_ERROR, result.getFunctionError());
                 session.transfer(flowFile, REL_FAILURE);
             } else {
@@ -188,31 +175,23 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
                 final long totalTimeMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
                 session.getProvenanceReporter().send(flowFile, functionName, totalTimeMillis);
             }
-        } catch (final InvalidRequestContentException
-            | InvalidParameterValueException
-            | RequestTooLargeException
-            | ResourceNotFoundException
-            | UnsupportedMediaTypeException unrecoverableException) {
-                getLogger().error("Failed to invoke lambda {} with unrecoverable exception {} for flow file {}",
-                    new Object[]{functionName, unrecoverableException, flowFile});
-                flowFile = populateExceptionAttributes(session, flowFile, unrecoverableException);
-                session.transfer(flowFile, REL_FAILURE);
+        } catch (final InvalidRequestContentException | InvalidParameterValueException | RequestTooLargeException | ResourceNotFoundException | UnsupportedMediaTypeException unrecoverableException) {
+            getLogger().error("Failed to invoke lambda {} with unrecoverable exception {} for flow file {}", functionName, unrecoverableException, flowFile);
+            flowFile = populateExceptionAttributes(session, flowFile, unrecoverableException);
+            session.transfer(flowFile, REL_FAILURE);
         } catch (final TooManyRequestsException retryableServiceException) {
-            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}, therefore penalizing flowfile",
-                new Object[]{functionName, retryableServiceException, flowFile});
+            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}, therefore penalizing flowfile", functionName, retryableServiceException, flowFile);
             flowFile = populateExceptionAttributes(session, flowFile, retryableServiceException);
             flowFile = session.penalize(flowFile);
             session.transfer(flowFile, REL_FAILURE);
             context.yield();
         } catch (final AmazonServiceException unrecoverableServiceException) {
-            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {} sending to fail",
-                new Object[]{functionName, unrecoverableServiceException, flowFile});
+            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {} sending to fail", functionName, unrecoverableServiceException, flowFile);
             flowFile = populateExceptionAttributes(session, flowFile, unrecoverableServiceException);
             session.transfer(flowFile, REL_FAILURE);
             context.yield();
         } catch (final Exception exception) {
-            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}",
-                new Object[]{functionName, exception, flowFile});
+            getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}", functionName, exception, flowFile);
             session.transfer(flowFile, REL_FAILURE);
             context.yield();
         }
@@ -227,12 +206,12 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
      */
     private FlowFile populateExceptionAttributes(final ProcessSession session, FlowFile flowFile,
             final AmazonServiceException exception) {
-        Map<String,String> attributes = new HashMap<>();
+        final Map<String, String> attributes = new HashMap<>();
         attributes.put(AWS_LAMBDA_EXCEPTION_MESSAGE, exception.getErrorMessage());
         attributes.put(AWS_LAMBDA_EXCEPTION_ERROR_CODE, exception.getErrorCode());
         attributes.put(AWS_LAMBDA_EXCEPTION_REQUEST_ID, exception.getRequestId());
         attributes.put(AWS_LAMBDA_EXCEPTION_STATUS_CODE, Integer.toString(exception.getStatusCode()));
-        if ( exception.getCause() != null )
+        if (exception.getCause() != null)
             attributes.put(AWS_LAMBDA_EXCEPTION_CAUSE, exception.getCause().getMessage());
         attributes.put(AWS_LAMBDA_EXCEPTION_ERROR_TYPE, exception.getErrorType().toString());
         attributes.put(AWS_LAMBDA_EXCEPTION_MESSAGE, exception.getErrorMessage());
@@ -240,4 +219,15 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
         return flowFile;
     }
 
+
+    @Override
+    protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                           final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+        return (AWSLambdaClient) AWSLambdaClient.builder()
+                .withRegion(region.getName())
+                .withEndpointConfiguration(endpointConfiguration)
+                .withCredentials(credentialsProvider)
+                .withClientConfiguration(config)
+                .build();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java
index 8419bee547..27f0664370 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java
@@ -17,26 +17,18 @@
 
 package org.apache.nifi.processors.aws.ml;
 
-import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
-
 import com.amazonaws.AmazonWebServiceClient;
 import com.amazonaws.AmazonWebServiceRequest;
 import com.amazonaws.AmazonWebServiceResult;
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.regions.Regions;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.MapperFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.json.JsonMapper;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.ExpressionLanguageScope;
@@ -48,6 +40,14 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+
 public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
         extends AbstractAWSCredentialsProviderProcessor<T> {
     public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
@@ -74,21 +74,20 @@ public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceCli
             .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
             .autoTerminateDefault(true)
             .build();
-    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+    protected static final List<PropertyDescriptor> PROPERTIES = List.of(
             MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
             REGION,
             TIMEOUT,
             JSON_PAYLOAD,
             SSL_CONTEXT_SERVICE,
-            ENDPOINT_OVERRIDE));
+            ENDPOINT_OVERRIDE);
+
     private final static ObjectMapper MAPPER = JsonMapper.builder()
             .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
             .build();
-    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
-            REL_ORIGINAL,
+    private static final Set<Relationship> relationships = Set.of(REL_ORIGINAL,
             REL_SUCCESS,
-            REL_FAILURE
-    )));
+            REL_FAILURE);
 
     @Override
     public Set<Relationship> getRelationships() {
@@ -138,8 +137,9 @@ public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceCli
     }
 
     @Override
-    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
-        throw new UnsupportedOperationException("createClient(ProcessContext, AWSCredentials, ClientConfiguration) is not supported");
+    protected T createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                             final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+        throw new UnsupportedOperationException();
     }
 
     protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java
index 157314c9cf..5a5f8f02e2 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java
@@ -17,23 +17,18 @@
 
 package org.apache.nifi.processors.aws.ml;
 
-import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
-
 import com.amazonaws.AmazonWebServiceClient;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.ResponseMetadata;
-import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
 import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Region;
 import com.amazonaws.regions.Regions;
 import com.fasterxml.jackson.databind.MapperFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.json.JsonMapper;
 import com.fasterxml.jackson.databind.module.SimpleModule;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
@@ -42,6 +37,11 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
 public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
         extends AbstractAWSCredentialsProviderProcessor<T>  {
     public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
@@ -89,14 +89,14 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
             .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
             .build();
     public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
-    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+    protected static final List<PropertyDescriptor> PROPERTIES = List.of(
             TASK_ID,
             MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
             REGION,
             TIMEOUT,
             SSL_CONTEXT_SERVICE,
             ENDPOINT_OVERRIDE,
-            PROXY_CONFIGURATION_SERVICE));
+            PROXY_CONFIGURATION_SERVICE);
     private static final ObjectMapper MAPPER = JsonMapper.builder()
             .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
             .build();
@@ -116,13 +116,13 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
         return relationships;
     }
 
-    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+    private static final Set<Relationship> relationships = Set.of(
             REL_ORIGINAL,
             REL_SUCCESS,
             REL_RUNNING,
             REL_THROTTLED,
             REL_FAILURE
-    )));
+    );
 
     @Override
     public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@@ -130,8 +130,9 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
     }
 
     @Override
-    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
-        throw new UnsupportedOperationException("Client creation not supported");
+    protected T createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                             final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+        throw new UnsupportedOperationException();
     }
 
     protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java
index 1ef4d604a4..73d6bfa148 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java
@@ -19,14 +19,13 @@ package org.apache.nifi.processors.aws.ml.polly;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.polly.AmazonPollyClient;
-import com.amazonaws.services.polly.AmazonPollyClientBuilder;
 import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
 import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
 import com.amazonaws.services.polly.model.TaskStatus;
 import com.amazonaws.services.textract.model.ThrottlingException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -38,6 +37,9 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
 
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 @Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
 @CapabilityDescription("Retrieves the current status of an AWS Polly job.")
 @SeeAlso({StartAwsPollyJob.class})
@@ -54,10 +56,13 @@ public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<A
     private static final String AWS_S3_KEY = "filename";
 
     @Override
-    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
-        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+    protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                             final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+        return (AmazonPollyClient) AmazonPollyClient.builder()
                 .withCredentials(credentialsProvider)
                 .withRegion(context.getProperty(REGION).getValue())
+                .withEndpointConfiguration(endpointConfiguration)
+                .withClientConfiguration(config)
                 .build();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java
index ba19878a59..0ab52ac597 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java
@@ -19,8 +19,9 @@ package org.apache.nifi.processors.aws.ml.polly;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.polly.AmazonPollyClient;
-import com.amazonaws.services.polly.AmazonPollyClientBuilder;
 import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
 import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -34,11 +35,16 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
 @CapabilityDescription("Trigger a AWS Polly job. It should be followed by GetAwsPollyJobStatus processor in order to monitor job status.")
 @SeeAlso({GetAwsPollyJobStatus.class})
 public class StartAwsPollyJob extends AwsMachineLearningJobStarter<AmazonPollyClient, StartSpeechSynthesisTaskRequest, StartSpeechSynthesisTaskResult> {
+
     @Override
-    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
-        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+    protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                             final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+
+        return (AmazonPollyClient) AmazonPollyClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
                 .withCredentials(credentialsProvider)
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .build();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java
index 07638fd335..ce48142f0c 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java
@@ -17,20 +17,16 @@
 
 package org.apache.nifi.processors.aws.ml.textract;
 
-import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
-
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.textract.AmazonTextractClient;
 import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
 import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
 import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
 import com.amazonaws.services.textract.model.JobStatus;
 import com.amazonaws.services.textract.model.ThrottlingException;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -43,6 +39,13 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
+
 @Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
 @CapabilityDescription("Retrieves the current status of an AWS Textract job.")
 @SeeAlso({StartAwsTextractJob.class})
@@ -66,9 +69,12 @@ public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcesso
     }
 
     @Override
-    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTextractClient) AmazonTextractClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .withCredentials(credentialsProvider)
                 .build();
     }
@@ -99,44 +105,25 @@ public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcesso
         } catch (ThrottlingException e) {
             getLogger().info("Request Rate Limit exceeded", e);
             session.transfer(flowFile, REL_THROTTLED);
-            return;
         } catch (Exception e) {
             getLogger().warn("Failed to get Textract Job status", e);
             session.transfer(flowFile, REL_FAILURE);
-            return;
         }
     }
 
     private Object getTask(TextractType typeOfTextract, AmazonTextractClient client, String awsTaskId) {
-        Object job = null;
-        switch (typeOfTextract) {
-            case DOCUMENT_ANALYSIS:
-                job = client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId));
-                break;
-            case DOCUMENT_TEXT_DETECTION:
-                job = client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId));
-                break;
-            case EXPENSE_ANALYSIS:
-                job = client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId));
-                break;
-        }
-        return job;
+        return switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS -> client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId));
+            case DOCUMENT_TEXT_DETECTION -> client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId));
+            case EXPENSE_ANALYSIS -> client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId));
+        };
     }
 
     private JobStatus getTaskStatus(TextractType typeOfTextract, AmazonTextractClient client, String awsTaskId) {
-        JobStatus jobStatus = JobStatus.IN_PROGRESS;
-        switch (typeOfTextract) {
-            case DOCUMENT_ANALYSIS:
-                jobStatus = JobStatus.fromValue(client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
-                break;
-            case DOCUMENT_TEXT_DETECTION:
-                jobStatus = JobStatus.fromValue(client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId)).getJobStatus());
-                break;
-            case EXPENSE_ANALYSIS:
-                jobStatus = JobStatus.fromValue(client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
-                break;
-
-        }
-        return jobStatus;
+        return switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS -> JobStatus.fromValue(client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
+            case DOCUMENT_TEXT_DETECTION -> JobStatus.fromValue(client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId)).getJobStatus());
+            case EXPENSE_ANALYSIS -> JobStatus.fromValue(client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
+        };
     }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java
index d67c81bb37..70ff2391a8 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java
@@ -17,12 +17,12 @@
 
 package org.apache.nifi.processors.aws.ml.textract;
 
-import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
-
 import com.amazonaws.AmazonWebServiceRequest;
 import com.amazonaws.AmazonWebServiceResult;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.textract.AmazonTextractClient;
 import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
 import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
@@ -30,10 +30,6 @@ import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
 import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
 import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
 import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -47,6 +43,13 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
+
 @Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
 @CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
 @SeeAlso({GetAwsTextractJobStatus.class})
@@ -86,69 +89,43 @@ public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonText
     }
 
     @Override
-    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTextractClient) AmazonTextractClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
                 .withCredentials(credentialsProvider)
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .build();
     }
 
     @Override
     protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context, FlowFile flowFile) {
-        TextractType textractType =
-                TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
-        AmazonWebServiceResult result;
-        switch (textractType) {
-            case DOCUMENT_ANALYSIS :
-                result = getClient(context).startDocumentAnalysis((StartDocumentAnalysisRequest) request);
-                break;
-            case DOCUMENT_TEXT_DETECTION:
-                result = getClient(context).startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
-                break;
-            case EXPENSE_ANALYSIS:
-                result = getClient(context).startExpenseAnalysis((StartExpenseAnalysisRequest) request);
-                break;
-            default: throw new UnsupportedOperationException("Unsupported textract type: " + textractType);
-        }
-        return result;
+        TextractType textractType = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
+        return switch (textractType) {
+            case DOCUMENT_ANALYSIS -> getClient(context).startDocumentAnalysis((StartDocumentAnalysisRequest) request);
+            case DOCUMENT_TEXT_DETECTION -> getClient(context).startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
+            case EXPENSE_ANALYSIS -> getClient(context).startExpenseAnalysis((StartExpenseAnalysisRequest) request);
+        };
     }
 
     @Override
     protected Class<? extends AmazonWebServiceRequest> getAwsRequestClass(ProcessContext context, FlowFile flowFile) {
-        TextractType typeOfTextract =
-                TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
-        Class<? extends AmazonWebServiceRequest> result = null;
-        switch (typeOfTextract) {
-            case DOCUMENT_ANALYSIS:
-                result = StartDocumentAnalysisRequest.class;
-                break;
-            case DOCUMENT_TEXT_DETECTION:
-                result = StartDocumentTextDetectionRequest.class;
-                break;
-            case EXPENSE_ANALYSIS:
-                result = StartExpenseAnalysisRequest.class;
-                break;
-        }
-        return result;
+        final TextractType typeOfTextract = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
+        return switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS -> StartDocumentAnalysisRequest.class;
+            case DOCUMENT_TEXT_DETECTION -> StartDocumentTextDetectionRequest.class;
+            case EXPENSE_ANALYSIS -> StartExpenseAnalysisRequest.class;
+        };
     }
 
     @Override
     protected String getAwsTaskId(ProcessContext context, AmazonWebServiceResult amazonWebServiceResult, FlowFile flowFile) {
-        TextractType textractType =
-                TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
-        String  result;
-        switch (textractType) {
-            case DOCUMENT_ANALYSIS:
-                result = ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
-                break;
-            case DOCUMENT_TEXT_DETECTION:
-                result = ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
-                break;
-            case EXPENSE_ANALYSIS:
-                result = ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
-                break;
-            default: throw new UnsupportedOperationException("Unsupported textract type.");
-        }
-        return result;
+        final TextractType textractType = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
+        return switch (textractType) {
+            case DOCUMENT_ANALYSIS -> ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
+            case DOCUMENT_TEXT_DETECTION -> ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
+            case EXPENSE_ANALYSIS -> ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
+        };
     }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java
index d19cd27a58..ff48af3843 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java
@@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.transcribe;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.textract.model.ThrottlingException;
 import com.amazonaws.services.transcribe.AmazonTranscribeClient;
 import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
@@ -42,11 +44,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
         @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
 })
 public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTranscribeClient> {
+
     @Override
-    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                  final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
                 .withCredentials(credentialsProvider)
+                .withEndpointConfiguration(endpointConfiguration)
+                .withClientConfiguration(config)
                 .build();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/StartAwsTranscribeJob.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/StartAwsTranscribeJob.java
index a6491394fb..a91192e5c6 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/StartAwsTranscribeJob.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/StartAwsTranscribeJob.java
@@ -18,8 +18,9 @@
 package org.apache.nifi.processors.aws.ml.transcribe;
 
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.transcribe.AmazonTranscribeClient;
 import com.amazonaws.services.transcribe.model.StartTranscriptionJobRequest;
 import com.amazonaws.services.transcribe.model.StartTranscriptionJobResult;
@@ -34,19 +35,18 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
 @CapabilityDescription("Trigger a AWS Transcribe job. It should be followed by GetAwsTranscribeStatus processor in order to monitor job status.")
 @SeeAlso({GetAwsTranscribeJobStatus.class})
 public class StartAwsTranscribeJob extends AwsMachineLearningJobStarter<AmazonTranscribeClient, StartTranscriptionJobRequest, StartTranscriptionJobResult> {
+
     @Override
-    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                  final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .withCredentials(credentialsProvider)
                 .build();
     }
 
-    @Override
-    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
-        return (AmazonTranscribeClient) AmazonTranscribeClient.builder().build();
-    }
-
     @Override
     protected StartTranscriptionJobResult sendRequest(StartTranscriptionJobRequest request, ProcessContext context, FlowFile flowFile) {
         return getClient(context).startTranscriptionJob(request);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java
index f26080977e..bc52a23efd 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java
@@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.translate;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.textract.model.ThrottlingException;
 import com.amazonaws.services.translate.AmazonTranslateClient;
 import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
@@ -42,11 +44,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
         @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
 })
 public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTranslateClient> {
+
     @Override
-    protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                 final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTranslateClient) AmazonTranslateClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
                 .withCredentials(credentialsProvider)
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .build();
     }
 
@@ -76,11 +82,9 @@ public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusProcess
         } catch (ThrottlingException e) {
             getLogger().info("Request Rate Limit exceeded", e);
             session.transfer(flowFile, REL_THROTTLED);
-            return;
         } catch (Exception e) {
             getLogger().warn("Failed to get Polly Job status", e);
             session.transfer(flowFile, REL_FAILURE);
-            return;
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/StartAwsTranslateJob.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/StartAwsTranslateJob.java
index 306d5e4ad9..3e7f079df9 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/StartAwsTranslateJob.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/StartAwsTranslateJob.java
@@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.translate;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.translate.AmazonTranslateClient;
 import com.amazonaws.services.translate.model.StartTextTranslationJobRequest;
 import com.amazonaws.services.translate.model.StartTextTranslationJobResult;
@@ -33,11 +35,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
 @CapabilityDescription("Trigger a AWS Translate job. It should be followed by GetAwsTranslateJobStatus processor in order to monitor job status.")
 @SeeAlso({GetAwsTranslateJobStatus.class})
 public class StartAwsTranslateJob extends AwsMachineLearningJobStarter<AmazonTranslateClient, StartTextTranslationJobRequest, StartTextTranslationJobResult> {
+
     @Override
-    protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+    protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                 final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
         return (AmazonTranslateClient) AmazonTranslateClient.builder()
                 .withRegion(context.getProperty(REGION).getValue())
                 .withCredentials(credentialsProvider)
+                .withClientConfiguration(config)
+                .withEndpointConfiguration(endpointConfiguration)
                 .build();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
index b93d7ebc4a..8210e99abe 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/DeleteS3Object.java
@@ -63,13 +63,13 @@ public class DeleteS3Object extends AbstractS3Processor {
             .build();
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
+            BUCKET_WITH_DEFAULT_VALUE,
             KEY,
-            BUCKET,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            S3_REGION,
             ACCESS_KEY,
             SECRET_KEY,
             CREDENTIALS_FILE,
-            AWS_CREDENTIALS_PROVIDER_SERVICE,
-            S3_REGION,
             TIMEOUT,
             VERSION_ID,
             FULL_CONTROL_USER_LIST,
@@ -114,7 +114,7 @@ public class DeleteS3Object extends AbstractS3Processor {
 
         final long startNanos = System.nanoTime();
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
         final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue();
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
index 69e61fb51a..56e6204810 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
@@ -214,13 +214,13 @@ public class FetchS3Object extends AbstractS3Processor {
             .build();
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
-            BUCKET,
+            BUCKET_WITH_DEFAULT_VALUE,
             KEY,
             S3_REGION,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
             ACCESS_KEY,
             SECRET_KEY,
             CREDENTIALS_FILE,
-            AWS_CREDENTIALS_PROVIDER_SERVICE,
             TIMEOUT,
             VERSION_ID,
             SSL_CONTEXT_SERVICE,
@@ -268,7 +268,7 @@ public class FetchS3Object extends AbstractS3Processor {
     public List<ConfigVerificationResult> verify(ProcessContext context, ComponentLog verificationLogger, Map<String, String> attributes) {
         final List<ConfigVerificationResult> results = new ArrayList<>(super.verify(context, verificationLogger, attributes));
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
 
         final AmazonS3Client client = createClient(context, attributes);
@@ -319,7 +319,7 @@ public class FetchS3Object extends AbstractS3Processor {
         if (encryptionService != null) {
             attributes.put("s3.encryptionStrategy", encryptionService.getStrategyName());
         }
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
 
         final GetObjectRequest request = createGetObjectRequest(context, flowFile.getAttributes());
@@ -333,7 +333,7 @@ public class FetchS3Object extends AbstractS3Processor {
 
             final ObjectMetadata metadata = s3Object.getObjectMetadata();
             if (metadata.getContentDisposition() != null) {
-                final String contentDisposition = URLDecoder.decode(metadata.getContentDisposition(), StandardCharsets.UTF_8.name());
+                final String contentDisposition = URLDecoder.decode(metadata.getContentDisposition(), StandardCharsets.UTF_8);
 
                 if (contentDisposition.equals(PutS3Object.CONTENT_DISPOSITION_INLINE) || contentDisposition.startsWith("attachment; filename=")) {
                     setFilePathAttributes(attributes, key);
@@ -401,7 +401,7 @@ public class FetchS3Object extends AbstractS3Processor {
     }
 
     private GetObjectMetadataRequest createGetObjectMetadataRequest(final ProcessContext context, final Map<String, String> attributes) {
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
         final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(attributes).getValue();
         final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();
@@ -417,7 +417,7 @@ public class FetchS3Object extends AbstractS3Processor {
     }
 
     private GetObjectRequest createGetObjectRequest(final ProcessContext context, final Map<String, String> attributes) {
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
         final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(attributes).getValue();
         final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
index 63077049e5..d79be0fcfa 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/ListS3.java
@@ -64,8 +64,8 @@ import org.apache.nifi.logging.ComponentLog;
 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.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processor.util.list.ListableEntityWrapper;
 import org.apache.nifi.processor.util.list.ListedEntity;
@@ -285,22 +285,22 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
 
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
+            BUCKET_WITHOUT_DEFAULT_VALUE,
+            REGION,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
             LISTING_STRATEGY,
             TRACKING_STATE_CACHE,
             INITIAL_LISTING_TARGET,
             TRACKING_TIME_WINDOW,
-            BUCKET,
-            REGION,
-            ACCESS_KEY,
-            SECRET_KEY,
             RECORD_WRITER,
             MIN_AGE,
             MAX_AGE,
             BATCH_SIZE,
             WRITE_OBJECT_TAGS,
             WRITE_USER_METADATA,
+            ACCESS_KEY,
+            SECRET_KEY,
             CREDENTIALS_FILE,
-            AWS_CREDENTIALS_PROVIDER_SERVICE,
             TIMEOUT,
             SSL_CONTEXT_SERVICE,
             ENDPOINT_OVERRIDE,
@@ -482,7 +482,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
         final Long maxAgeMilliseconds = context.getProperty(MAX_AGE) != null ? context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS) : null;
         final long listingTimestamp = System.currentTimeMillis();
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
+        final String bucket = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions().getValue();
         final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
 
         final ListingSnapshot currentListing = listing.get();
@@ -703,7 +703,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
         final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();
         final boolean useVersions = context.getProperty(USE_VERSIONS).asBoolean();
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
+        final String bucket = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions().getValue();
         final String delimiter = context.getProperty(DELIMITER).getValue();
         final String prefix = context.getProperty(PREFIX).evaluateAttributeExpressions().getValue();
 
@@ -1138,7 +1138,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
         final AmazonS3Client client = createClient(context);
 
         final List<ConfigVerificationResult> results = new ArrayList<>(super.verify(context, logger, attributes));
-        final String bucketName = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
+        final String bucketName = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
         final long minAgeMilliseconds = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
         final Long maxAgeMilliseconds = context.getProperty(MAX_AGE) != null ? context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS) : null;
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
index 732b0ff968..f66c6ba021 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java
@@ -16,35 +16,29 @@
  */
 package org.apache.nifi.processors.aws.s3;
 
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-import java.net.URLEncoder;
-import java.nio.file.Files;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.stream.Collectors;
-
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
+import com.amazonaws.services.s3.model.AccessControlList;
+import com.amazonaws.services.s3.model.AmazonS3Exception;
+import com.amazonaws.services.s3.model.CannedAccessControlList;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
+import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.amazonaws.services.s3.model.MultipartUploadListing;
+import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.ObjectTagging;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.StorageClass;
 import com.amazonaws.services.s3.model.Tag;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -68,52 +62,42 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
-import com.amazonaws.services.s3.model.AccessControlList;
-import com.amazonaws.services.s3.model.AmazonS3Exception;
-import com.amazonaws.services.s3.model.CannedAccessControlList;
-import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
-import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
-import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
-import com.amazonaws.services.s3.model.MultipartUpload;
-import com.amazonaws.services.s3.model.MultipartUploadListing;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.PartETag;
-import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
-import com.amazonaws.services.s3.model.StorageClass;
-import com.amazonaws.services.s3.model.UploadPartRequest;
-import com.amazonaws.services.s3.model.UploadPartResult;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
 
 @SupportsBatching
 @SeeAlso({FetchS3Object.class, DeleteS3Object.class, ListS3.class})
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"Amazon", "S3", "AWS", "Archive", "Put"})
-@CapabilityDescription("Puts FlowFiles to an Amazon S3 Bucket.\n" +
-        "The upload uses either the PutS3Object method or the PutS3MultipartUpload method.  The PutS3Object method " +
-        "sends the file in a single synchronous call, but it has a 5GB size limit.  Larger files are sent using the " +
-        "PutS3MultipartUpload method.  This multipart process " +
-        "saves state after each step so that a large upload can be resumed with minimal loss if the processor or " +
-        "cluster is stopped and restarted.\n" +
-        "A multipart upload consists of three steps:\n" +
-        "  1) initiate upload,\n" +
-        "  2) upload the parts, and\n" +
-        "  3) complete the upload.\n" +
-        "For multipart uploads, the processor saves state locally tracking the upload ID and parts uploaded, which " +
-        "must both be provided to complete the upload.\n" +
-        "The AWS libraries select an endpoint URL based on the AWS region, but this can be overridden with the " +
-        "'Endpoint Override URL' property for use with other S3-compatible endpoints.\n" +
-        "The S3 API specifies that the maximum file size for a PutS3Object upload is 5GB. It also requires that " +
-        "parts in a multipart upload must be at least 5MB in size, except for the last part.  These limits " +
-        "establish the bounds for the Multipart Upload Threshold and Part Size properties.")
+@CapabilityDescription("Writes the contents of a FlowFile as an S3 Object to an Amazon S3 Bucket.")
 @DynamicProperty(name = "The name of a User-Defined Metadata field to add to the S3 Object",
-value = "The value of a User-Defined Metadata field to add to the S3 Object",
-description = "Allows user-defined metadata to be added to the S3 object as key/value pairs",
-expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        value = "The value of a User-Defined Metadata field to add to the S3 Object",
+        description = "Allows user-defined metadata to be added to the S3 object as key/value pairs",
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
 @ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the S3 object")
 @WritesAttributes({
     @WritesAttribute(attribute = "s3.bucket", description = "The S3 bucket where the Object was put in S3"),
@@ -277,19 +261,21 @@ public class PutS3Object extends AbstractS3Processor {
             .build();
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
+            BUCKET_WITH_DEFAULT_VALUE,
             KEY,
-            BUCKET,
+            S3_REGION,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            STORAGE_CLASS,
+            ENCRYPTION_SERVICE,
+            SERVER_SIDE_ENCRYPTION,
             CONTENT_TYPE,
             CONTENT_DISPOSITION,
             CACHE_CONTROL,
             ACCESS_KEY,
             SECRET_KEY,
             CREDENTIALS_FILE,
-            AWS_CREDENTIALS_PROVIDER_SERVICE,
             OBJECT_TAGS_PREFIX,
             REMOVE_TAG_PREFIX,
-            STORAGE_CLASS,
-            S3_REGION,
             TIMEOUT,
             EXPIRATION_RULE_ID,
             FULL_CONTROL_USER_LIST,
@@ -309,8 +295,6 @@ public class PutS3Object extends AbstractS3Processor {
             MULTIPART_S3_AGEOFF_INTERVAL,
             MULTIPART_S3_MAX_AGE,
             MULTIPART_TEMP_DIR,
-            SERVER_SIDE_ENCRYPTION,
-            ENCRYPTION_SERVICE,
             USE_CHUNKED_ENCODING,
             USE_PATH_STYLE_ACCESS,
             PROXY_CONFIGURATION_SERVICE,
@@ -366,7 +350,7 @@ public class PutS3Object extends AbstractS3Processor {
         return new File(this.tempDirMultipart + File.separator + getIdentifier());
     }
 
-    protected boolean localUploadExistsInS3(final AmazonS3Client s3, final String bucket, final MultipartState localState) {
+    protected boolean localUploadExistsInS3(final AmazonS3 s3, final String bucket, final MultipartState localState) {
         ListMultipartUploadsRequest listRequest = new ListMultipartUploadsRequest(bucket);
         MultipartUploadListing listing = s3.listMultipartUploads(listRequest);
 
@@ -378,7 +362,7 @@ public class PutS3Object extends AbstractS3Processor {
         return false;
     }
 
-    protected synchronized MultipartState getLocalStateIfInS3(final AmazonS3Client s3, final String bucket,
+    protected synchronized MultipartState getLocalStateIfInS3(final AmazonS3 s3, final String bucket,
             final String s3ObjectKey) throws IOException {
         MultipartState currState = getLocalState(s3ObjectKey);
         if (currState == null) {
@@ -522,7 +506,7 @@ public class PutS3Object extends AbstractS3Processor {
 
         final long startNanos = System.nanoTime();
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
         final String cacheKey = getIdentifier() + "/" + bucket + "/" + key;
 
@@ -549,325 +533,323 @@ public class PutS3Object extends AbstractS3Processor {
             final FlowFile flowFileCopy = flowFile;
             session.read(flowFile, new InputStreamCallback() {
                 @Override
-                public void process(final InputStream rawIn) throws IOException {
-                    try (final InputStream in = new BufferedInputStream(rawIn)) {
-                        final ObjectMetadata objectMetadata = new ObjectMetadata();
-                        objectMetadata.setContentLength(ff.getSize());
-
-                        final String contentType = context.getProperty(CONTENT_TYPE)
-                                .evaluateAttributeExpressions(ff).getValue();
-                        if (contentType != null) {
-                            objectMetadata.setContentType(contentType);
-                            attributes.put(S3_CONTENT_TYPE, contentType);
-                        }
+                public void process(final InputStream in) throws IOException {
+                    final ObjectMetadata objectMetadata = new ObjectMetadata();
+                    objectMetadata.setContentLength(ff.getSize());
+
+                    final String contentType = context.getProperty(CONTENT_TYPE)
+                            .evaluateAttributeExpressions(ff).getValue();
+                    if (contentType != null) {
+                        objectMetadata.setContentType(contentType);
+                        attributes.put(S3_CONTENT_TYPE, contentType);
+                    }
+
+                    final String cacheControl = context.getProperty(CACHE_CONTROL)
+                            .evaluateAttributeExpressions(ff).getValue();
+                    if (cacheControl != null) {
+                        objectMetadata.setCacheControl(cacheControl);
+                        attributes.put(S3_CACHE_CONTROL, cacheControl);
+                    }
+
+                    final String contentDisposition = context.getProperty(CONTENT_DISPOSITION).getValue();
+                    String fileName = URLEncoder.encode(ff.getAttribute(CoreAttributes.FILENAME.key()), StandardCharsets.UTF_8);
+                    if (contentDisposition != null && contentDisposition.equals(CONTENT_DISPOSITION_INLINE)) {
+                        objectMetadata.setContentDisposition(CONTENT_DISPOSITION_INLINE);
+                        attributes.put(S3_CONTENT_DISPOSITION, CONTENT_DISPOSITION_INLINE);
+                    } else if (contentDisposition != null && contentDisposition.equals(CONTENT_DISPOSITION_ATTACHMENT)) {
+                        String contentDispositionValue = CONTENT_DISPOSITION_ATTACHMENT + "; filename=\"" + fileName + "\"";
+                        objectMetadata.setContentDisposition(contentDispositionValue);
+                        attributes.put(S3_CONTENT_DISPOSITION, contentDispositionValue);
+                    } else {
+                        objectMetadata.setContentDisposition(fileName);
+                    }
+
+                    final String expirationRule = context.getProperty(EXPIRATION_RULE_ID)
+                            .evaluateAttributeExpressions(ff).getValue();
+                    if (expirationRule != null) {
+                        objectMetadata.setExpirationTimeRuleId(expirationRule);
+                    }
 
-                        final String cacheControl = context.getProperty(CACHE_CONTROL)
-                                .evaluateAttributeExpressions(ff).getValue();
-                        if (cacheControl != null) {
-                            objectMetadata.setCacheControl(cacheControl);
-                            attributes.put(S3_CACHE_CONTROL, cacheControl);
+                    final Map<String, String> userMetadata = new HashMap<>();
+                    for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
+                        if (entry.getKey().isDynamic()) {
+                            final String value = context.getProperty(
+                                    entry.getKey()).evaluateAttributeExpressions(ff).getValue();
+                            userMetadata.put(entry.getKey().getName(), value);
                         }
+                    }
 
-                        final String contentDisposition = context.getProperty(CONTENT_DISPOSITION).getValue();
-                        String fileName = URLEncoder.encode(ff.getAttribute(CoreAttributes.FILENAME.key()), "UTF-8");
-                        if (contentDisposition != null && contentDisposition.equals(CONTENT_DISPOSITION_INLINE)) {
-                            objectMetadata.setContentDisposition(CONTENT_DISPOSITION_INLINE);
-                            attributes.put(S3_CONTENT_DISPOSITION, CONTENT_DISPOSITION_INLINE);
-                        } else if (contentDisposition != null && contentDisposition.equals(CONTENT_DISPOSITION_ATTACHMENT)) {
-                            String contentDispositionValue = CONTENT_DISPOSITION_ATTACHMENT + "; filename=\"" + fileName + "\"";
-                            objectMetadata.setContentDisposition(contentDispositionValue);
-                            attributes.put(S3_CONTENT_DISPOSITION, contentDispositionValue);
-                        } else {
-                            objectMetadata.setContentDisposition(fileName);
+                    final String serverSideEncryption = context.getProperty(SERVER_SIDE_ENCRYPTION).getValue();
+                    AmazonS3EncryptionService encryptionService = null;
+
+                    if (!serverSideEncryption.equals(NO_SERVER_SIDE_ENCRYPTION)) {
+                        objectMetadata.setSSEAlgorithm(serverSideEncryption);
+                        attributes.put(S3_SSE_ALGORITHM, serverSideEncryption);
+                    } else {
+                        encryptionService = context.getProperty(ENCRYPTION_SERVICE).asControllerService(AmazonS3EncryptionService.class);
+                    }
+
+                    if (!userMetadata.isEmpty()) {
+                        objectMetadata.setUserMetadata(userMetadata);
+                    }
+
+                    if (ff.getSize() <= multipartThreshold) {
+                        //----------------------------------------
+                        // single part upload
+                        //----------------------------------------
+                        final PutObjectRequest request = new PutObjectRequest(bucket, key, in, objectMetadata);
+                        if (encryptionService != null) {
+                            encryptionService.configurePutObjectRequest(request, objectMetadata);
+                            attributes.put(S3_ENCRYPTION_STRATEGY, encryptionService.getStrategyName());
                         }
 
-                        final String expirationRule = context.getProperty(EXPIRATION_RULE_ID)
-                                .evaluateAttributeExpressions(ff).getValue();
-                        if (expirationRule != null) {
-                            objectMetadata.setExpirationTimeRuleId(expirationRule);
+                        request.setStorageClass(StorageClass.valueOf(context.getProperty(STORAGE_CLASS).getValue()));
+                        final AccessControlList acl = createACL(context, ff);
+                        if (acl != null) {
+                            request.setAccessControlList(acl);
                         }
 
-                        final Map<String, String> userMetadata = new HashMap<>();
-                        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
-                            if (entry.getKey().isDynamic()) {
-                                final String value = context.getProperty(
-                                        entry.getKey()).evaluateAttributeExpressions(ff).getValue();
-                                userMetadata.put(entry.getKey().getName(), value);
-                            }
+                        final CannedAccessControlList cannedAcl = createCannedACL(context, ff);
+                        if (cannedAcl != null) {
+                            request.withCannedAcl(cannedAcl);
                         }
 
-                        final String serverSideEncryption = context.getProperty(SERVER_SIDE_ENCRYPTION).getValue();
-                        AmazonS3EncryptionService encryptionService = null;
+                        if (context.getProperty(OBJECT_TAGS_PREFIX).isSet()) {
+                            request.setTagging(new ObjectTagging(getObjectTags(context, flowFileCopy)));
+                        }
 
-                        if (!serverSideEncryption.equals(NO_SERVER_SIDE_ENCRYPTION)) {
-                            objectMetadata.setSSEAlgorithm(serverSideEncryption);
-                            attributes.put(S3_SSE_ALGORITHM, serverSideEncryption);
-                        } else {
-                            encryptionService = context.getProperty(ENCRYPTION_SERVICE).asControllerService(AmazonS3EncryptionService.class);
+                        try {
+                            final PutObjectResult result = s3.putObject(request);
+                            if (result.getVersionId() != null) {
+                                attributes.put(S3_VERSION_ATTR_KEY, result.getVersionId());
+                            }
+                            if (result.getETag() != null) {
+                                attributes.put(S3_ETAG_ATTR_KEY, result.getETag());
+                            }
+                            if (result.getExpirationTime() != null) {
+                                attributes.put(S3_EXPIRATION_ATTR_KEY, result.getExpirationTime().toString());
+                            }
+                            if (result.getMetadata().getStorageClass() != null) {
+                                attributes.put(S3_STORAGECLASS_ATTR_KEY, result.getMetadata().getStorageClass());
+                            } else {
+                                attributes.put(S3_STORAGECLASS_ATTR_KEY, StorageClass.Standard.toString());
+                            }
+                            if (userMetadata.size() > 0) {
+                                StringBuilder userMetaBldr = new StringBuilder();
+                                for (String userKey : userMetadata.keySet()) {
+                                    userMetaBldr.append(userKey).append("=").append(userMetadata.get(userKey));
+                                }
+                                attributes.put(S3_USERMETA_ATTR_KEY, userMetaBldr.toString());
+                            }
+                            attributes.put(S3_API_METHOD_ATTR_KEY, S3_API_METHOD_PUTOBJECT);
+                        } catch (AmazonClientException e) {
+                            getLogger().info("Failure completing upload flowfile={} bucket={} key={} reason={}",
+                                    ffFilename, bucket, key, e.getMessage());
+                            throw (e);
                         }
+                    } else {
+                        //----------------------------------------
+                        // multipart upload
+                        //----------------------------------------
 
-                        if (!userMetadata.isEmpty()) {
-                            objectMetadata.setUserMetadata(userMetadata);
+                        // load or create persistent state
+                        //------------------------------------------------------------
+                        MultipartState currentState;
+                        try {
+                            currentState = getLocalStateIfInS3(s3, bucket, cacheKey);
+                            if (currentState != null) {
+                                if (currentState.getPartETags().size() > 0) {
+                                    final PartETag lastETag = currentState.getPartETags().get(
+                                            currentState.getPartETags().size() - 1);
+                                    getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
+                                                    "uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
+                                                    "contentLength='{}' partsLoaded={} lastPart={}/{}",
+                                            ffFilename, bucket, key, currentState.getUploadId(),
+                                            currentState.getFilePosition(), currentState.getPartSize(),
+                                            currentState.getStorageClass().toString(),
+                                            currentState.getContentLength(),
+                                            currentState.getPartETags().size(),
+                                            Integer.toString(lastETag.getPartNumber()),
+                                            lastETag.getETag());
+                                } else {
+                                    getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
+                                                    "uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
+                                                    "contentLength='{}' no partsLoaded",
+                                            ffFilename, bucket, key, currentState.getUploadId(),
+                                            currentState.getFilePosition(), currentState.getPartSize(),
+                                            currentState.getStorageClass().toString(),
+                                            currentState.getContentLength());
+                                }
+                            } else {
+                                currentState = new MultipartState();
+                                currentState.setPartSize(multipartPartSize);
+                                currentState.setStorageClass(
+                                        StorageClass.valueOf(context.getProperty(STORAGE_CLASS).getValue()));
+                                currentState.setContentLength(ff.getSize());
+                                persistLocalState(cacheKey, currentState);
+                                getLogger().info("Starting new upload for flowfile='{}' bucket='{}' key='{}'",
+                                        ffFilename, bucket, key);
+                            }
+                        } catch (IOException e) {
+                            getLogger().error("IOException initiating cache state while processing flow files: " +
+                                    e.getMessage());
+                            throw (e);
                         }
 
-                        if (ff.getSize() <= multipartThreshold) {
-                            //----------------------------------------
-                            // single part upload
-                            //----------------------------------------
-                            final PutObjectRequest request = new PutObjectRequest(bucket, key, in, objectMetadata);
+                        // initiate multipart upload or find position in file
+                        //------------------------------------------------------------
+                        if (currentState.getUploadId().isEmpty()) {
+                            final InitiateMultipartUploadRequest initiateRequest = new InitiateMultipartUploadRequest(bucket, key, objectMetadata);
                             if (encryptionService != null) {
-                                encryptionService.configurePutObjectRequest(request, objectMetadata);
+                                encryptionService.configureInitiateMultipartUploadRequest(initiateRequest, objectMetadata);
                                 attributes.put(S3_ENCRYPTION_STRATEGY, encryptionService.getStrategyName());
                             }
+                            initiateRequest.setStorageClass(currentState.getStorageClass());
 
-                            request.setStorageClass(StorageClass.valueOf(context.getProperty(STORAGE_CLASS).getValue()));
                             final AccessControlList acl = createACL(context, ff);
                             if (acl != null) {
-                                request.setAccessControlList(acl);
+                                initiateRequest.setAccessControlList(acl);
                             }
-
                             final CannedAccessControlList cannedAcl = createCannedACL(context, ff);
                             if (cannedAcl != null) {
-                                request.withCannedAcl(cannedAcl);
+                                initiateRequest.withCannedACL(cannedAcl);
                             }
 
                             if (context.getProperty(OBJECT_TAGS_PREFIX).isSet()) {
-                                request.setTagging(new ObjectTagging(getObjectTags(context, flowFileCopy)));
+                                initiateRequest.setTagging(new ObjectTagging(getObjectTags(context, flowFileCopy)));
                             }
 
                             try {
-                                final PutObjectResult result = s3.putObject(request);
-                                if (result.getVersionId() != null) {
-                                    attributes.put(S3_VERSION_ATTR_KEY, result.getVersionId());
-                                }
-                                if (result.getETag() != null) {
-                                    attributes.put(S3_ETAG_ATTR_KEY, result.getETag());
-                                }
-                                if (result.getExpirationTime() != null) {
-                                    attributes.put(S3_EXPIRATION_ATTR_KEY, result.getExpirationTime().toString());
-                                }
-                                if (result.getMetadata().getStorageClass() != null) {
-                                    attributes.put(S3_STORAGECLASS_ATTR_KEY, result.getMetadata().getStorageClass());
-                                } else {
-                                    attributes.put(S3_STORAGECLASS_ATTR_KEY, StorageClass.Standard.toString());
+                                final InitiateMultipartUploadResult initiateResult =
+                                        s3.initiateMultipartUpload(initiateRequest);
+                                currentState.setUploadId(initiateResult.getUploadId());
+                                currentState.getPartETags().clear();
+                                try {
+                                    persistLocalState(cacheKey, currentState);
+                                } catch (Exception e) {
+                                    getLogger().info("Exception saving cache state while processing flow file: " +
+                                            e.getMessage());
+                                    throw (new ProcessException("Exception saving cache state", e));
                                 }
-                                if (userMetadata.size() > 0) {
-                                    StringBuilder userMetaBldr = new StringBuilder();
-                                    for (String userKey : userMetadata.keySet()) {
-                                        userMetaBldr.append(userKey).append("=").append(userMetadata.get(userKey));
-                                    }
-                                    attributes.put(S3_USERMETA_ATTR_KEY, userMetaBldr.toString());
+                                getLogger().info("Success initiating upload flowfile={} available={} position={} " +
+                                                "length={} bucket={} key={} uploadId={}",
+                                        new Object[]{ffFilename, in.available(), currentState.getFilePosition(),
+                                                currentState.getContentLength(), bucket, key,
+                                                currentState.getUploadId()});
+                                if (initiateResult.getUploadId() != null) {
+                                    attributes.put(S3_UPLOAD_ID_ATTR_KEY, initiateResult.getUploadId());
                                 }
-                                attributes.put(S3_API_METHOD_ATTR_KEY, S3_API_METHOD_PUTOBJECT);
                             } catch (AmazonClientException e) {
-                                getLogger().info("Failure completing upload flowfile={} bucket={} key={} reason={}",
+                                getLogger().info("Failure initiating upload flowfile={} bucket={} key={} reason={}",
                                         new Object[]{ffFilename, bucket, key, e.getMessage()});
                                 throw (e);
                             }
                         } else {
-                            //----------------------------------------
-                            // multipart upload
-                            //----------------------------------------
-
-                            // load or create persistent state
-                            //------------------------------------------------------------
-                            MultipartState currentState;
-                            try {
-                                currentState = getLocalStateIfInS3(s3, bucket, cacheKey);
-                                if (currentState != null) {
-                                    if (currentState.getPartETags().size() > 0) {
-                                        final PartETag lastETag = currentState.getPartETags().get(
-                                                currentState.getPartETags().size() - 1);
-                                        getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
-                                                "uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
-                                                "contentLength='{}' partsLoaded={} lastPart={}/{}",
-                                                new Object[]{ffFilename, bucket, key, currentState.getUploadId(),
-                                                        currentState.getFilePosition(), currentState.getPartSize(),
-                                                        currentState.getStorageClass().toString(),
-                                                        currentState.getContentLength(),
-                                                        currentState.getPartETags().size(),
-                                                        Integer.toString(lastETag.getPartNumber()),
-                                                        lastETag.getETag()});
-                                    } else {
-                                        getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
-                                                "uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
-                                                "contentLength='{}' no partsLoaded",
-                                                new Object[]{ffFilename, bucket, key, currentState.getUploadId(),
-                                                        currentState.getFilePosition(), currentState.getPartSize(),
-                                                        currentState.getStorageClass().toString(),
-                                                        currentState.getContentLength()});
-                                    }
-                                } else {
-                                    currentState = new MultipartState();
-                                    currentState.setPartSize(multipartPartSize);
-                                    currentState.setStorageClass(
-                                            StorageClass.valueOf(context.getProperty(STORAGE_CLASS).getValue()));
-                                    currentState.setContentLength(ff.getSize());
-                                    persistLocalState(cacheKey, currentState);
-                                    getLogger().info("Starting new upload for flowfile='{}' bucket='{}' key='{}'",
-                                            new Object[]{ffFilename, bucket, key});
-                                }
-                            } catch (IOException e) {
-                                getLogger().error("IOException initiating cache state while processing flow files: " +
-                                        e.getMessage());
-                                throw (e);
-                            }
-
-                            // initiate multipart upload or find position in file
-                            //------------------------------------------------------------
-                            if (currentState.getUploadId().isEmpty()) {
-                                final InitiateMultipartUploadRequest initiateRequest = new InitiateMultipartUploadRequest(bucket, key, objectMetadata);
-                                if (encryptionService != null) {
-                                    encryptionService.configureInitiateMultipartUploadRequest(initiateRequest, objectMetadata);
-                                    attributes.put(S3_ENCRYPTION_STRATEGY, encryptionService.getStrategyName());
-                                }
-                                initiateRequest.setStorageClass(currentState.getStorageClass());
-
-                                final AccessControlList acl = createACL(context, ff);
-                                if (acl != null) {
-                                    initiateRequest.setAccessControlList(acl);
-                                }
-                                final CannedAccessControlList cannedAcl = createCannedACL(context, ff);
-                                if (cannedAcl != null) {
-                                    initiateRequest.withCannedACL(cannedAcl);
-                                }
-
-                                if (context.getProperty(OBJECT_TAGS_PREFIX).isSet()) {
-                                    initiateRequest.setTagging(new ObjectTagging(getObjectTags(context, flowFileCopy)));
-                                }
-
+                            if (currentState.getFilePosition() > 0) {
                                 try {
-                                    final InitiateMultipartUploadResult initiateResult =
-                                            s3.initiateMultipartUpload(initiateRequest);
-                                    currentState.setUploadId(initiateResult.getUploadId());
-                                    currentState.getPartETags().clear();
-                                    try {
-                                        persistLocalState(cacheKey, currentState);
-                                    } catch (Exception e) {
-                                        getLogger().info("Exception saving cache state while processing flow file: " +
-                                                e.getMessage());
-                                        throw(new ProcessException("Exception saving cache state", e));
-                                    }
-                                    getLogger().info("Success initiating upload flowfile={} available={} position={} " +
-                                            "length={} bucket={} key={} uploadId={}",
-                                            new Object[]{ffFilename, in.available(), currentState.getFilePosition(),
-                                                    currentState.getContentLength(), bucket, key,
-                                                    currentState.getUploadId()});
-                                    if (initiateResult.getUploadId() != null) {
-                                        attributes.put(S3_UPLOAD_ID_ATTR_KEY, initiateResult.getUploadId());
-                                    }
-                                } catch (AmazonClientException e) {
-                                    getLogger().info("Failure initiating upload flowfile={} bucket={} key={} reason={}",
-                                            new Object[]{ffFilename, bucket, key, e.getMessage()});
-                                    throw(e);
-                                }
-                            } else {
-                                if (currentState.getFilePosition() > 0) {
-                                    try {
-                                        final long skipped = in.skip(currentState.getFilePosition());
-                                        if (skipped != currentState.getFilePosition()) {
-                                            getLogger().info("Failure skipping to resume upload flowfile={} " +
-                                                    "bucket={} key={} position={} skipped={}",
-                                                    new Object[]{ffFilename, bucket, key,
-                                                            currentState.getFilePosition(), skipped});
-                                        }
-                                    } catch (Exception e) {
-                                        getLogger().info("Failure skipping to resume upload flowfile={} bucket={} " +
-                                                "key={} position={} reason={}",
-                                                new Object[]{ffFilename, bucket, key, currentState.getFilePosition(),
-                                                        e.getMessage()});
-                                        throw(new ProcessException(e));
+                                    final long skipped = in.skip(currentState.getFilePosition());
+                                    if (skipped != currentState.getFilePosition()) {
+                                        getLogger().info("Failure skipping to resume upload flowfile={} " +
+                                                        "bucket={} key={} position={} skipped={}",
+                                                new Object[]{ffFilename, bucket, key,
+                                                        currentState.getFilePosition(), skipped});
                                     }
+                                } catch (Exception e) {
+                                    getLogger().info("Failure skipping to resume upload flowfile={} bucket={} " +
+                                                    "key={} position={} reason={}",
+                                            new Object[]{ffFilename, bucket, key, currentState.getFilePosition(),
+                                                    e.getMessage()});
+                                    throw (new ProcessException(e));
                                 }
                             }
+                        }
 
-                            // upload parts
-                            //------------------------------------------------------------
-                            long thisPartSize;
-                            boolean isLastPart;
-                            for (int part = currentState.getPartETags().size() + 1;
-                                    currentState.getFilePosition() < currentState.getContentLength(); part++) {
-                                if (!PutS3Object.this.isScheduled()) {
-                                    throw new IOException(S3_PROCESS_UNSCHEDULED_MESSAGE + " flowfile=" + ffFilename +
-                                            " part=" + part + " uploadId=" + currentState.getUploadId());
-                                }
-                                thisPartSize = Math.min(currentState.getPartSize(),
-                                        (currentState.getContentLength() - currentState.getFilePosition()));
-                                isLastPart = currentState.getContentLength() == currentState.getFilePosition() + thisPartSize;
-                                UploadPartRequest uploadRequest = new UploadPartRequest()
-                                        .withBucketName(bucket)
-                                        .withKey(key)
-                                        .withUploadId(currentState.getUploadId())
-                                        .withInputStream(in)
-                                        .withPartNumber(part)
-                                        .withPartSize(thisPartSize)
-                                        .withLastPart(isLastPart);
-                                if (encryptionService != null) {
-                                    encryptionService.configureUploadPartRequest(uploadRequest, objectMetadata);
+                        // upload parts
+                        //------------------------------------------------------------
+                        long thisPartSize;
+                        boolean isLastPart;
+                        for (int part = currentState.getPartETags().size() + 1;
+                             currentState.getFilePosition() < currentState.getContentLength(); part++) {
+                            if (!PutS3Object.this.isScheduled()) {
+                                throw new IOException(S3_PROCESS_UNSCHEDULED_MESSAGE + " flowfile=" + ffFilename +
+                                        " part=" + part + " uploadId=" + currentState.getUploadId());
+                            }
+                            thisPartSize = Math.min(currentState.getPartSize(),
+                                    (currentState.getContentLength() - currentState.getFilePosition()));
+                            isLastPart = currentState.getContentLength() == currentState.getFilePosition() + thisPartSize;
+                            UploadPartRequest uploadRequest = new UploadPartRequest()
+                                    .withBucketName(bucket)
+                                    .withKey(key)
+                                    .withUploadId(currentState.getUploadId())
+                                    .withInputStream(in)
+                                    .withPartNumber(part)
+                                    .withPartSize(thisPartSize)
+                                    .withLastPart(isLastPart);
+                            if (encryptionService != null) {
+                                encryptionService.configureUploadPartRequest(uploadRequest, objectMetadata);
+                            }
+                            try {
+                                UploadPartResult uploadPartResult = s3.uploadPart(uploadRequest);
+                                currentState.addPartETag(uploadPartResult.getPartETag());
+                                currentState.setFilePosition(currentState.getFilePosition() + thisPartSize);
+                                try {
+                                    persistLocalState(cacheKey, currentState);
+                                } catch (Exception e) {
+                                    getLogger().info("Exception saving cache state processing flow file: " +
+                                            e.getMessage());
                                 }
+                                int available = 0;
                                 try {
-                                    UploadPartResult uploadPartResult = s3.uploadPart(uploadRequest);
-                                    currentState.addPartETag(uploadPartResult.getPartETag());
-                                    currentState.setFilePosition(currentState.getFilePosition() + thisPartSize);
-                                    try {
-                                        persistLocalState(cacheKey, currentState);
-                                    } catch (Exception e) {
-                                        getLogger().info("Exception saving cache state processing flow file: " +
-                                                e.getMessage());
-                                    }
-                                    int available = 0;
-                                    try {
-                                        available = in.available();
-                                    } catch (IOException e) {
-                                        // in case of the last part, the stream is already closed
-                                    }
-                                    getLogger().info("Success uploading part flowfile={} part={} available={} " +
-                                            "etag={} uploadId={}", new Object[]{ffFilename, part, available,
-                                                    uploadPartResult.getETag(), currentState.getUploadId()});
-                                } catch (AmazonClientException e) {
-                                    getLogger().info("Failure uploading part flowfile={} part={} bucket={} key={} " +
-                                            "reason={}", new Object[]{ffFilename, part, bucket, key, e.getMessage()});
-                                    throw (e);
+                                    available = in.available();
+                                } catch (IOException e) {
+                                    // in case of the last part, the stream is already closed
                                 }
+                                getLogger().info("Success uploading part flowfile={} part={} available={} " +
+                                        "etag={} uploadId={}", new Object[]{ffFilename, part, available,
+                                        uploadPartResult.getETag(), currentState.getUploadId()});
+                            } catch (AmazonClientException e) {
+                                getLogger().info("Failure uploading part flowfile={} part={} bucket={} key={} " +
+                                        "reason={}", new Object[]{ffFilename, part, bucket, key, e.getMessage()});
+                                throw (e);
                             }
+                        }
 
-                            // complete multipart upload
-                            //------------------------------------------------------------
-                            CompleteMultipartUploadRequest completeRequest = new CompleteMultipartUploadRequest(
-                                    bucket, key, currentState.getUploadId(), currentState.getPartETags());
+                        // complete multipart upload
+                        //------------------------------------------------------------
+                        CompleteMultipartUploadRequest completeRequest = new CompleteMultipartUploadRequest(
+                                bucket, key, currentState.getUploadId(), currentState.getPartETags());
 
-                            // No call to an encryption service is needed for a CompleteMultipartUploadRequest.
-                            try {
-                                CompleteMultipartUploadResult completeResult =
-                                        s3.completeMultipartUpload(completeRequest);
-                                getLogger().info("Success completing upload flowfile={} etag={} uploadId={}",
-                                        new Object[]{ffFilename, completeResult.getETag(), currentState.getUploadId()});
-                                if (completeResult.getVersionId() != null) {
-                                    attributes.put(S3_VERSION_ATTR_KEY, completeResult.getVersionId());
-                                }
-                                if (completeResult.getETag() != null) {
-                                    attributes.put(S3_ETAG_ATTR_KEY, completeResult.getETag());
-                                }
-                                if (completeResult.getExpirationTime() != null) {
-                                    attributes.put(S3_EXPIRATION_ATTR_KEY,
-                                            completeResult.getExpirationTime().toString());
-                                }
-                                if (currentState.getStorageClass() != null) {
-                                    attributes.put(S3_STORAGECLASS_ATTR_KEY, currentState.getStorageClass().toString());
-                                }
-                                if (userMetadata.size() > 0) {
-                                    StringBuilder userMetaBldr = new StringBuilder();
-                                    for (String userKey : userMetadata.keySet()) {
-                                        userMetaBldr.append(userKey).append("=").append(userMetadata.get(userKey));
-                                    }
-                                    attributes.put(S3_USERMETA_ATTR_KEY, userMetaBldr.toString());
+                        // No call to an encryption service is needed for a CompleteMultipartUploadRequest.
+                        try {
+                            CompleteMultipartUploadResult completeResult =
+                                    s3.completeMultipartUpload(completeRequest);
+                            getLogger().info("Success completing upload flowfile={} etag={} uploadId={}",
+                                    new Object[]{ffFilename, completeResult.getETag(), currentState.getUploadId()});
+                            if (completeResult.getVersionId() != null) {
+                                attributes.put(S3_VERSION_ATTR_KEY, completeResult.getVersionId());
+                            }
+                            if (completeResult.getETag() != null) {
+                                attributes.put(S3_ETAG_ATTR_KEY, completeResult.getETag());
+                            }
+                            if (completeResult.getExpirationTime() != null) {
+                                attributes.put(S3_EXPIRATION_ATTR_KEY,
+                                        completeResult.getExpirationTime().toString());
+                            }
+                            if (currentState.getStorageClass() != null) {
+                                attributes.put(S3_STORAGECLASS_ATTR_KEY, currentState.getStorageClass().toString());
+                            }
+                            if (userMetadata.size() > 0) {
+                                StringBuilder userMetaBldr = new StringBuilder();
+                                for (String userKey : userMetadata.keySet()) {
+                                    userMetaBldr.append(userKey).append("=").append(userMetadata.get(userKey));
                                 }
-                                attributes.put(S3_API_METHOD_ATTR_KEY, S3_API_METHOD_MULTIPARTUPLOAD);
-                            } catch (AmazonClientException e) {
-                                getLogger().info("Failure completing upload flowfile={} bucket={} key={} reason={}",
-                                        new Object[]{ffFilename, bucket, key, e.getMessage()});
-                                throw (e);
+                                attributes.put(S3_USERMETA_ATTR_KEY, userMetaBldr.toString());
                             }
+                            attributes.put(S3_API_METHOD_ATTR_KEY, S3_API_METHOD_MULTIPARTUPLOAD);
+                        } catch (AmazonClientException e) {
+                            getLogger().info("Failure completing upload flowfile={} bucket={} key={} reason={}",
+                                    new Object[]{ffFilename, bucket, key, e.getMessage()});
+                            throw (e);
                         }
                     }
                 }
@@ -907,20 +889,20 @@ public class PutS3Object extends AbstractS3Processor {
     private final AtomicLong lastS3AgeOff = new AtomicLong(0L);
     private final DateFormat logFormat = new SimpleDateFormat();
 
-    protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
+    protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3 s3, final long now, String bucket) {
         MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now, bucket);
         for (MultipartUpload upload : oldUploads.getMultipartUploads()) {
             abortS3MultipartUpload(s3, oldUploads.getBucketName(), upload);
         }
     }
 
-    protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
-        final long ageoff_interval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+    protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3 s3, final long now, String bucket) {
+        final long ageoffInterval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
         final Long maxAge = context.getProperty(MULTIPART_S3_MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
         final long ageCutoff = now - maxAge;
 
         final List<MultipartUpload> ageoffList = new ArrayList<>();
-        if ((lastS3AgeOff.get() < now - ageoff_interval) && s3BucketLock.tryLock()) {
+        if ((lastS3AgeOff.get() < now - ageoffInterval) && s3BucketLock.tryLock()) {
             try {
 
                 ListMultipartUploadsRequest listRequest = new ListMultipartUploadsRequest(bucket);
@@ -959,7 +941,7 @@ public class PutS3Object extends AbstractS3Processor {
         return result;
     }
 
-    protected void abortS3MultipartUpload(final AmazonS3Client s3, final String bucket, final MultipartUpload upload) {
+    protected void abortS3MultipartUpload(final AmazonS3 s3, final String bucket, final MultipartUpload upload) {
         final String uploadKey = upload.getKey();
         final String uploadId = upload.getUploadId();
         final AbortMultipartUploadRequest abortRequest = new AbortMultipartUploadRequest(
@@ -1106,7 +1088,7 @@ public class PutS3Object extends AbstractS3Processor {
                     } else {
                         first = false;
                     }
-                    buf.append(String.format("%d/%s", tag.getPartNumber(), tag.getETag()));
+                    buf.append(java.lang.String.format("%d/%s", tag.getPartNumber(), tag.getETag()));
                 }
             }
             buf.append(SEPARATOR)
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java
index 9a79839b68..d8620ddc54 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/TagS3Object.java
@@ -62,8 +62,7 @@ import java.util.stream.Collectors;
 @SeeAlso({PutS3Object.class, FetchS3Object.class, ListS3.class})
 @Tags({"Amazon", "S3", "AWS", "Archive", "Tag"})
 @InputRequirement(Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Sets tags on a FlowFile within an Amazon S3 Bucket. " +
-        "If attempting to tag a file that does not exist, FlowFile is routed to success.")
+@CapabilityDescription("Adds or updates a tag on an Amazon S3 Object.")
 public class TagS3Object extends AbstractS3Processor {
 
     public static final PropertyDescriptor TAG_KEY = new PropertyDescriptor.Builder()
@@ -107,17 +106,17 @@ public class TagS3Object extends AbstractS3Processor {
             .build();
 
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
+            BUCKET_WITH_DEFAULT_VALUE,
             KEY,
-            BUCKET,
-            VERSION_ID,
+            S3_REGION,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
             TAG_KEY,
             TAG_VALUE,
             APPEND_TAG,
+            VERSION_ID,
             ACCESS_KEY,
             SECRET_KEY,
             CREDENTIALS_FILE,
-            AWS_CREDENTIALS_PROVIDER_SERVICE,
-            S3_REGION,
             TIMEOUT,
             SSL_CONTEXT_SERVICE,
             ENDPOINT_OVERRIDE,
@@ -155,13 +154,13 @@ public class TagS3Object extends AbstractS3Processor {
 
         final long startNanos = System.nanoTime();
 
-        final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
+        final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
         final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
         final String newTagKey = context.getProperty(TAG_KEY).evaluateAttributeExpressions(flowFile).getValue();
         final String newTagVal = context.getProperty(TAG_VALUE).evaluateAttributeExpressions(flowFile).getValue();
 
         if(StringUtils.isBlank(bucket)){
-            failFlowWithBlankEvaluatedProperty(session, flowFile, BUCKET);
+            failFlowWithBlankEvaluatedProperty(session, flowFile, BUCKET_WITH_DEFAULT_VALUE);
             return;
         }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java
index 45fdc5e4e1..0e69c87d82 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java
@@ -16,10 +16,13 @@
  */
 package org.apache.nifi.processors.aws.s3.encryption;
 
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.AmazonS3EncryptionClient;
+import com.amazonaws.regions.Region;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Builder;
+import com.amazonaws.services.s3.AmazonS3EncryptionClientV2;
+import com.amazonaws.services.s3.AmazonS3EncryptionClientV2Builder;
+import com.amazonaws.services.s3.model.CryptoConfigurationV2;
 import com.amazonaws.services.s3.model.EncryptionMaterials;
 import com.amazonaws.services.s3.model.StaticEncryptionMaterialsProvider;
 import org.apache.commons.codec.binary.Base64;
@@ -27,6 +30,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.components.ValidationResult;
 
 import javax.crypto.spec.SecretKeySpec;
+import java.util.function.Consumer;
 
 /**
  * This strategy uses a client master key to perform client-side encryption.   Use this strategy when you want the client to perform the encryption,
@@ -39,26 +43,31 @@ public class ClientSideCEncryptionStrategy implements S3EncryptionStrategy {
     /**
      * Create an encryption client.
      *
-     * @param credentialsProvider AWS credentials provider.
-     * @param clientConfiguration Client configuration
+     * @param clientBuilder A consumer that is responsible for configuring the client builder
      * @param kmsRegion not used by this encryption strategy
      * @param keyIdOrMaterial client master key, always base64 encoded
      * @return AWS S3 client
      */
     @Override
-    public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
-        ValidationResult keyValidationResult = validateKey(keyIdOrMaterial);
+    public AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder, final String kmsRegion, final String keyIdOrMaterial) {
+        final ValidationResult keyValidationResult = validateKey(keyIdOrMaterial);
         if (!keyValidationResult.isValid()) {
             throw new IllegalArgumentException("Invalid client key; " + keyValidationResult.getExplanation());
         }
 
-        byte[] keyMaterial = Base64.decodeBase64(keyIdOrMaterial);
-        SecretKeySpec symmetricKey = new SecretKeySpec(keyMaterial, "AES");
-        StaticEncryptionMaterialsProvider encryptionMaterialsProvider = new StaticEncryptionMaterialsProvider(new EncryptionMaterials(symmetricKey));
+        final byte[] keyMaterial = Base64.decodeBase64(keyIdOrMaterial);
+        final SecretKeySpec symmetricKey = new SecretKeySpec(keyMaterial, "AES");
+        final StaticEncryptionMaterialsProvider encryptionMaterialsProvider = new StaticEncryptionMaterialsProvider(new EncryptionMaterials(symmetricKey));
 
-        AmazonS3EncryptionClient client = new AmazonS3EncryptionClient(credentialsProvider, encryptionMaterialsProvider);
+        final CryptoConfigurationV2 cryptoConfig = new CryptoConfigurationV2();
+        cryptoConfig.setAwsKmsRegion(Region.getRegion(Regions.fromName(kmsRegion)));
 
-        return client;
+        final AmazonS3EncryptionClientV2Builder builder = AmazonS3EncryptionClientV2.encryptionBuilder()
+                .disableChunkedEncoding()
+                .withCryptoConfiguration(cryptoConfig)
+                .withEncryptionMaterialsProvider(encryptionMaterialsProvider);
+        clientBuilder.accept(builder);
+        return builder.build();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java
deleted file mode 100644
index 1da5dbbf56..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.aws.s3.encryption;
-
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.AmazonS3EncryptionClient;
-import com.amazonaws.services.s3.model.CryptoConfiguration;
-import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider;
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * This strategy uses KMS key id to perform client-side encryption.  Use this strategy when you want the client to perform the encryption,
- * (thus incurring the cost of processing) and manage the key in a KMS instance.
- *
- * See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingClientSideEncryption.html#client-side-encryption-kms-managed-master-key-intro
- *
- */
-public class ClientSideKMSEncryptionStrategy implements S3EncryptionStrategy {
-    /**
-     * Create an encryption client.
-     *
-     * @param credentialsProvider AWS credentials provider.
-     * @param clientConfiguration Client configuration
-     * @param kmsRegion AWS KMS region
-     * @param keyIdOrMaterial KMS key id
-     * @return AWS S3 client
-     */
-    @Override
-    public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
-        KMSEncryptionMaterialsProvider materialProvider = new KMSEncryptionMaterialsProvider(keyIdOrMaterial);
-        boolean haveKmsRegion = StringUtils.isNotBlank(kmsRegion);
-
-        CryptoConfiguration cryptoConfig = new CryptoConfiguration();
-        if (haveKmsRegion) {
-            Region awsRegion = Region.getRegion(Regions.fromName(kmsRegion));
-            cryptoConfig.setAwsKmsRegion(awsRegion);
-        }
-
-        AmazonS3EncryptionClient client = new AmazonS3EncryptionClient(credentialsProvider, materialProvider, cryptoConfig);
-
-        return client;
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/S3EncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/S3EncryptionStrategy.java
index a0012cbd40..967038f8f4 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/S3EncryptionStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/S3EncryptionStrategy.java
@@ -16,9 +16,8 @@
  */
 package org.apache.nifi.processors.aws.s3.encryption;
 
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Builder;
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ObjectMetadata;
@@ -26,6 +25,8 @@ import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import org.apache.nifi.components.ValidationResult;
 
+import java.util.function.Consumer;
+
 /**
  * This interface defines the API for S3 encryption strategies.  The methods have empty defaults
  * to minimize the burden on implementations.
@@ -72,11 +73,8 @@ public interface S3EncryptionStrategy {
     /**
      * Create an S3 encryption client.
      *
-     * @param credentialsProvider AWS credentials provider.
-     * @param clientConfiguration Client configuration.
-     * @return {@link AmazonS3Client}, perhaps an {@link com.amazonaws.services.s3.AmazonS3EncryptionClient}
      */
-    default AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
+    default AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder, String kmsRegion, String keyIdOrMaterial) {
         return null;
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideCEncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideCEncryptionStrategy.java
index 574675c43b..1939e19c93 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideCEncryptionStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideCEncryptionStrategy.java
@@ -30,8 +30,7 @@ import org.apache.nifi.components.ValidationResult;
  * This strategy uses a customer key to perform server-side encryption.  Use this strategy when you want the server to perform the encryption,
  * (meaning you pay cost of processing) and when you want to manage the key material yourself.
  *
- * See https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
- *
+ * See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html">Server Side Encryption Customer Keys</a>.
  */
 public class ServerSideCEncryptionStrategy implements S3EncryptionStrategy {
     @Override
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideKMSEncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideKMSEncryptionStrategy.java
index a31222e1a5..e2d9ea5b49 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideKMSEncryptionStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideKMSEncryptionStrategy.java
@@ -25,7 +25,7 @@ import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
  * This strategy uses a KMS key to perform server-side encryption.  Use this strategy when you want the server to perform the encryption,
  * (meaning you pay the cost of processing) and when you want to use a KMS key.
  *
- * See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+ * See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html">Using KMS Encryption</a>
  *
  */
 public class ServerSideKMSEncryptionStrategy implements S3EncryptionStrategy {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideS3EncryptionStrategy.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideS3EncryptionStrategy.java
index 89f1637aa3..e6d4f4367c 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideS3EncryptionStrategy.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ServerSideS3EncryptionStrategy.java
@@ -26,7 +26,7 @@ import com.amazonaws.services.s3.model.PutObjectRequest;
  * perform the encryption (meaning you pay the cost of processing) and you want AWS to completely manage the key.
  *
  *
- * See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+ * See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html">Using Server Side Encryption</a>
  *
  */
 public class ServerSideS3EncryptionStrategy implements S3EncryptionStrategy {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/StandardS3EncryptionService.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/StandardS3EncryptionService.java
index beac3661d5..a311e7e368 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/StandardS3EncryptionService.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/StandardS3EncryptionService.java
@@ -16,10 +16,9 @@
  */
 package org.apache.nifi.processors.aws.s3.encryption;
 
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.regions.Regions;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Builder;
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ObjectMetadata;
@@ -33,13 +32,12 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
-
-import org.apache.nifi.processors.aws.s3.AmazonS3EncryptionService;
 import org.apache.nifi.processors.aws.s3.AbstractS3Processor;
-
+import org.apache.nifi.processors.aws.s3.AmazonS3EncryptionService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
@@ -51,6 +49,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Consumer;
 
 
 @Tags({"service", "aws", "s3", "encryption", "encrypt", "decryption", "decrypt", "key"})
@@ -58,14 +57,13 @@ import java.util.Map;
 public class StandardS3EncryptionService extends AbstractControllerService implements AmazonS3EncryptionService {
     private static final Logger logger = LoggerFactory.getLogger(StandardS3EncryptionService.class);
 
-    private static final Map<String, S3EncryptionStrategy> NAMED_STRATEGIES = new HashMap<String, S3EncryptionStrategy>() {{
-        put(STRATEGY_NAME_NONE, new NoOpEncryptionStrategy());
-        put(STRATEGY_NAME_SSE_S3, new ServerSideS3EncryptionStrategy());
-        put(STRATEGY_NAME_SSE_KMS, new ServerSideKMSEncryptionStrategy());
-        put(STRATEGY_NAME_SSE_C, new ServerSideCEncryptionStrategy());
-        put(STRATEGY_NAME_CSE_KMS, new ClientSideKMSEncryptionStrategy());
-        put(STRATEGY_NAME_CSE_C, new ClientSideCEncryptionStrategy());
-    }};
+    private static final Map<String, S3EncryptionStrategy> NAMED_STRATEGIES = Map.of(
+            STRATEGY_NAME_NONE, new NoOpEncryptionStrategy(),
+            STRATEGY_NAME_SSE_S3, new ServerSideS3EncryptionStrategy(),
+            STRATEGY_NAME_SSE_KMS, new ServerSideKMSEncryptionStrategy(),
+            STRATEGY_NAME_SSE_C, new ServerSideCEncryptionStrategy(),
+            STRATEGY_NAME_CSE_C, new ClientSideCEncryptionStrategy()
+    );
 
     private static final AllowableValue NONE = new AllowableValue(STRATEGY_NAME_NONE, "None","No encryption.");
     private static final AllowableValue SSE_S3 = new AllowableValue(STRATEGY_NAME_SSE_S3, "Server-side S3","Use server-side, S3-managed encryption.");
@@ -100,7 +98,7 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
                     "In case of Server-side Customer Key, the key must be an AES-256 key. In case of Client-side Customer Key, it can be an AES-256, AES-192 or AES-128 key.")
             .required(false)
             .sensitive(true)
-            .addValidator((subject, input, context) -> new ValidationResult.Builder().valid(true).build()) // will be validated in customValidate()
+            .addValidator(Validator.VALID) // will be validated in customValidate()
             .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
             .build();
 
@@ -123,11 +121,8 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
         final String newStrategyName = context.getProperty(ENCRYPTION_STRATEGY).getValue();
         final String newKeyValue = context.getProperty(ENCRYPTION_VALUE).evaluateAttributeExpressions().getValue();
         final S3EncryptionStrategy newEncryptionStrategy = NAMED_STRATEGIES.get(newStrategyName);
-        String newKmsRegion = null;
 
-        if (context.getProperty(KMS_REGION) != null ) {
-            newKmsRegion = context.getProperty(KMS_REGION).getValue();
-        }
+        kmsRegion = context.getProperty(KMS_REGION).getValue();
 
         if (newEncryptionStrategy == null) {
             final String msg = "No encryption strategy found for name: " + strategyName;
@@ -138,7 +133,6 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
         strategyName = newStrategyName;
         encryptionStrategy = newEncryptionStrategy;
         keyValue = newKeyValue;
-        kmsRegion = newKmsRegion;
     }
 
     @Override
@@ -224,8 +218,8 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
     }
 
     @Override
-    public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration) {
-        return encryptionStrategy.createEncryptionClient(credentialsProvider, clientConfiguration, kmsRegion, keyValue);
+    public AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder) {
+        return encryptionStrategy.createEncryptionClient(clientBuilder, kmsRegion, keyValue);
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
index 43a1dec034..39285aba4a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java
@@ -24,22 +24,25 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.sqs.GetSQS;
 import org.apache.nifi.processors.aws.sqs.PutSQS;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.services.sns.SnsClient;
+import software.amazon.awssdk.services.sns.SnsClientBuilder;
 import software.amazon.awssdk.services.sns.model.MessageAttributeValue;
 import software.amazon.awssdk.services.sns.model.PublishRequest;
 
 import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -51,7 +54,7 @@ import java.util.Map;
 @CapabilityDescription("Sends the content of a FlowFile as a notification to the Amazon Simple Notification Service")
 @DynamicProperty(name = "A name of an attribute to be added to the notification", value = "The attribute value", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
         description = "User specified dynamic Properties are added as attributes to the notification")
-public class PutSNS extends AbstractSNSProcessor {
+public class PutSNS extends AbstractAwsSyncProcessor<SnsClient, SnsClientBuilder> {
 
     public static final PropertyDescriptor CHARACTER_ENCODING = new PropertyDescriptor.Builder()
             .name("Character Set")
@@ -61,6 +64,7 @@ public class PutSNS extends AbstractSNSProcessor {
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();
+
     public static final PropertyDescriptor USE_JSON_STRUCTURE = new PropertyDescriptor.Builder()
             .name("Use JSON Structure")
             .description("If true, the contents of the FlowFile must be JSON with a top-level element named 'default'."
@@ -70,6 +74,7 @@ public class PutSNS extends AbstractSNSProcessor {
             .allowableValues("true", "false")
             .required(true)
             .build();
+
     public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder()
             .name("E-mail Subject")
             .description("The optional subject to use for any subscribers that are subscribed via E-mail")
@@ -96,10 +101,45 @@ public class PutSNS extends AbstractSNSProcessor {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(ARN, ARN_TYPE, SUBJECT, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
-                    USE_JSON_STRUCTURE, CHARACTER_ENCODING, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD,
-                    MESSAGEGROUPID, MESSAGEDEDUPLICATIONID));
+
+    public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
+            .name("Amazon Resource Name (ARN)")
+            .description("The name of the resource to which notifications should be published")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    protected static final AllowableValue ARN_TYPE_TOPIC = new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic");
+    protected static final AllowableValue ARN_TYPE_TARGET = new AllowableValue("Target ARN", "Target ARN", "The ARN is the name of a particular Target, used to notify a specific subscriber");
+    public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
+            .name("ARN Type")
+            .description("The type of Amazon Resource Name that is being used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
+            .defaultValue(ARN_TYPE_TOPIC.getValue())
+            .build();
+
+
+    public static final List<PropertyDescriptor> properties = List.of(
+            ARN,
+            ARN_TYPE,
+            SUBJECT,
+            REGION,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            USE_JSON_STRUCTURE,
+            CHARACTER_ENCODING,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            MESSAGEGROUPID,
+            MESSAGEDEDUPLICATIONID);
 
     public static final int MAX_SIZE = 256 * 1024;
 
@@ -134,9 +174,13 @@ public class PutSNS extends AbstractSNSProcessor {
 
         final Charset charset = Charset.forName(context.getProperty(CHARACTER_ENCODING).evaluateAttributeExpressions(flowFile).getValue());
 
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        session.exportTo(flowFile, baos);
-        final String message = new String(baos.toByteArray(), charset);
+        final String message;
+        try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+            session.exportTo(flowFile, baos);
+            message = baos.toString(charset);
+        } catch (final IOException ioe) {
+            throw new ProcessException("Failed to read FlowFile content", ioe);
+        }
 
         final SnsClient client = getClient(context);
 
@@ -196,4 +240,9 @@ public class PutSNS extends AbstractSNSProcessor {
         }
     }
 
+    @Override
+    protected SnsClientBuilder createClientBuilder(final ProcessContext context) {
+        return SnsClient.builder();
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
index e5e3cc87b3..b03b2dc48f 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java
@@ -30,13 +30,13 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.services.sqs.SqsClient;
+import software.amazon.awssdk.services.sqs.SqsClientBuilder;
 import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest;
 import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry;
 import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse;
 
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
 
 @SupportsBatching
@@ -44,7 +44,15 @@ import java.util.List;
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"Amazon", "AWS", "SQS", "Queue", "Delete"})
 @CapabilityDescription("Deletes a message from an Amazon Simple Queuing Service Queue")
-public class DeleteSQS extends AbstractSQSProcessor {
+public class DeleteSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
+
+    public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
+            .name("Queue URL")
+            .description("The URL of the queue delete from")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
 
     public static final PropertyDescriptor RECEIPT_HANDLE = new PropertyDescriptor.Builder()
             .name("Receipt Handle")
@@ -55,9 +63,20 @@ public class DeleteSQS extends AbstractSQSProcessor {
             .defaultValue("${sqs.receipt.handle}")
             .build();
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(QUEUE_URL, RECEIPT_HANDLE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE,
-                    REGION, TIMEOUT, ENDPOINT_OVERRIDE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD));
+    public static final List<PropertyDescriptor> properties = List.of(
+            QUEUE_URL,
+            RECEIPT_HANDLE,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            ENDPOINT_OVERRIDE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD);
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@@ -101,8 +120,11 @@ public class DeleteSQS extends AbstractSQSProcessor {
             getLogger().error("Failed to delete message from SQS due to {}", new Object[] { e });
             flowFile = session.penalize(flowFile);
             session.transfer(flowFile, REL_FAILURE);
-            return;
         }
     }
 
+    @Override
+    protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
+        return SqsClient.builder();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
index 71391b38e4..cd90eb224a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java
@@ -26,14 +26,15 @@ 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.expression.AttributeExpression.ResultType;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.services.sqs.SqsClient;
+import software.amazon.awssdk.services.sqs.SqsClientBuilder;
 import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest;
 import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry;
 import software.amazon.awssdk.services.sqs.model.Message;
@@ -45,7 +46,6 @@ import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse;
 
 import java.nio.charset.Charset;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -64,7 +64,15 @@ import java.util.concurrent.TimeUnit;
     @WritesAttribute(attribute = "sqs.message.id", description = "The unique identifier of the SQS message"),
     @WritesAttribute(attribute = "sqs.receipt.handle", description = "The SQS Receipt Handle that is to be used to delete the message from the queue")
 })
-public class GetSQS extends AbstractSQSProcessor {
+public class GetSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
+
+    public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
+            .name("Queue URL")
+            .description("The URL of the queue to get messages from")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .required(true)
+            .build();
 
     public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
             .name("Character Set")
@@ -99,12 +107,6 @@ public class GetSQS extends AbstractSQSProcessor {
             .defaultValue("10")
             .build();
 
-    public static final PropertyDescriptor DYNAMIC_QUEUE_URL = new PropertyDescriptor.Builder()
-            .fromPropertyDescriptor(QUEUE_URL)
-            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
-            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
-            .build();
-
     public static final PropertyDescriptor RECEIVE_MSG_WAIT_TIME = new PropertyDescriptor.Builder()
             .name("Receive Message Wait Time")
             .description("The maximum amount of time to wait on a long polling receive call. Setting this to a value of 1 second or greater will "
@@ -115,11 +117,24 @@ public class GetSQS extends AbstractSQSProcessor {
             .addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.SECONDS, 20, TimeUnit.SECONDS))  // 20 seconds is the maximum allowed by SQS
             .build();
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(DYNAMIC_QUEUE_URL, AUTO_DELETE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
-                    AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, BATCH_SIZE, TIMEOUT, ENDPOINT_OVERRIDE,
-                    CHARSET, VISIBILITY_TIMEOUT, RECEIVE_MSG_WAIT_TIME, PROXY_HOST, PROXY_HOST_PORT,
-                    PROXY_USERNAME, PROXY_PASSWORD));
+    public static final List<PropertyDescriptor> properties = List.of(
+            QUEUE_URL,
+            AUTO_DELETE,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            BATCH_SIZE,
+            TIMEOUT,
+            ENDPOINT_OVERRIDE,
+            CHARSET,
+            VISIBILITY_TIMEOUT,
+            RECEIVE_MSG_WAIT_TIME,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD);
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@@ -133,9 +148,7 @@ public class GetSQS extends AbstractSQSProcessor {
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
-        final String queueUrl = context.getProperty(DYNAMIC_QUEUE_URL).evaluateAttributeExpressions()
-                .getValue();
-
+        final String queueUrl = context.getProperty(QUEUE_URL).evaluateAttributeExpressions().getValue();
         final SqsClient client = getClient(context);
 
         final ReceiveMessageRequest request = ReceiveMessageRequest.builder()
@@ -220,4 +233,10 @@ public class GetSQS extends AbstractSQSProcessor {
                 + " may be duplicated. Reason for deletion failure: {}", new Object[]{messages.size(), e});
         }
     }
+
+    @Override
+    protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
+        return SqsClient.builder();
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
index 6667673cbd..21fbf1580f 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java
@@ -32,7 +32,9 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
 import software.amazon.awssdk.services.sqs.SqsClient;
+import software.amazon.awssdk.services.sqs.SqsClientBuilder;
 import software.amazon.awssdk.services.sqs.model.MessageAttributeValue;
 import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest;
 import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry;
@@ -40,7 +42,6 @@ import software.amazon.awssdk.services.sqs.model.SendMessageBatchResponse;
 
 import java.io.ByteArrayOutputStream;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -55,9 +56,25 @@ import java.util.concurrent.TimeUnit;
 @DynamicProperty(name = "The name of a Message Attribute to add to the message", value = "The value of the Message Attribute",
         description = "Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of "
         + "the Message Attribute and value will become the value of the Message Attribute", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-public class PutSQS extends AbstractSQSProcessor {
+public class PutSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
     private static final String STRING_DATA_TYPE = "String";
 
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("Batch Size")
+            .description("The maximum number of messages to send in a single network request")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("25")
+            .build();
+
+    public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
+            .name("Queue URL")
+            .description("The URL of the queue to act upon")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .build();
+
     public static final PropertyDescriptor DELAY = new PropertyDescriptor.Builder()
             .name("Delay")
             .displayName("Delay")
@@ -85,10 +102,22 @@ public class PutSQS extends AbstractSQSProcessor {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
-    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
-            Arrays.asList(QUEUE_URL, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE,
-                    REGION, DELAY, TIMEOUT, ENDPOINT_OVERRIDE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME,
-                    PROXY_PASSWORD, MESSAGEGROUPID, MESSAGEDEDUPLICATIONID));
+    public static final List<PropertyDescriptor> properties = List.of(
+            QUEUE_URL,
+            ACCESS_KEY,
+            SECRET_KEY,
+            CREDENTIALS_FILE,
+            AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            DELAY,
+            TIMEOUT,
+            ENDPOINT_OVERRIDE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            MESSAGEGROUPID,
+            MESSAGEDEDUPLICATIONID);
 
     private volatile List<PropertyDescriptor> userDefinedProperties = Collections.emptyList();
 
@@ -158,7 +187,7 @@ public class PutSQS extends AbstractSQSProcessor {
                 .build();
 
         try {
-            SendMessageBatchResponse response = client.sendMessageBatch(request);
+            final SendMessageBatchResponse response = client.sendMessageBatch(request);
 
             // check for errors
             if (!response.failed().isEmpty()) {
@@ -177,4 +206,8 @@ public class PutSQS extends AbstractSQSProcessor {
         session.getProvenanceReporter().send(flowFile, queueUrl, transmissionMillis);
     }
 
+    @Override
+    protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
+        return SqsClient.builder();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/wag/InvokeAWSGatewayApi.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/wag/InvokeAWSGatewayApi.java
index fef63f6252..c77e9812fc 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/wag/InvokeAWSGatewayApi.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/wag/InvokeAWSGatewayApi.java
@@ -17,17 +17,6 @@
 package org.apache.nifi.processors.aws.wag;
 
 import com.amazonaws.http.AmazonHttpClient;
-import java.io.ByteArrayInputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 import org.apache.http.impl.EnglishReasonPhraseCatalog;
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -54,6 +43,18 @@ import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequest;
 import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayResponse;
 import org.apache.nifi.stream.io.StreamUtils;
 
+import java.io.ByteArrayInputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
 @SupportsBatching
 @InputRequirement(Requirement.INPUT_ALLOWED)
 @Tags({"Amazon", "AWS", "Client", "Gateway-API", "Rest", "http", "https"})
@@ -76,7 +77,7 @@ public class InvokeAWSGatewayApi extends AbstractAWSGatewayApiProcessor {
     public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays
             .asList(
                     PROP_METHOD,
-                    PROP_AWS_GATEWAY_API_REGION,
+                    REGION,
                     ACCESS_KEY,
                     SECRET_KEY,
                     CREDENTIALS_FILE,
@@ -249,17 +250,12 @@ public class InvokeAWSGatewayApi extends AbstractAWSGatewayApiProcessor {
                 // transfer the message body to the payload
                 // can potentially be null in edge cases
                 if (bodyExists) {
-                    final String contentType = response.getHttpResponse().getHeaders()
-                                                       .get("Content-Type");
-                    if (!(contentType == null) && !contentType.trim().isEmpty()) {
-                        responseFlowFile = session
-                            .putAttribute(responseFlowFile, CoreAttributes.MIME_TYPE.key(),
-                                          contentType.trim());
+                    final List<String> contentTypes = response.getHttpResponse().getHeaderValues("Content-Type");
+                    if (contentTypes != null && !contentTypes.isEmpty()) {
+                        responseFlowFile = session.putAttribute(responseFlowFile, CoreAttributes.MIME_TYPE.key(), contentTypes.get(0).trim());
                     }
 
-                    responseFlowFile = session
-                        .importFrom(new ByteArrayInputStream(response.getBody().getBytes()),
-                                    responseFlowFile);
+                    responseFlowFile = session.importFrom(new ByteArrayInputStream(response.getBody().getBytes()), responseFlowFile);
 
                     // emit provenance event
                     final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/docs/org.apache.nifi.processors.aws.s3.PutS3Object/additionalDetails.html b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/docs/org.apache.nifi.processors.aws.s3.PutS3Object/additionalDetails.html
index 50a0210cfb..df2f86ef8d 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/docs/org.apache.nifi.processors.aws.s3.PutS3Object/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/docs/org.apache.nifi.processors.aws.s3.PutS3Object/additionalDetails.html
@@ -21,6 +21,26 @@
     <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
 </head>
 <body>
+    <h2>Multi-part Upload Details</h2>
+    <p>
+        The upload uses either the PutS3Object method or the PutS3MultipartUpload method. The PutS3Object method sends the file in a single synchronous call,
+        but it has a 5GB size limit.  Larger files are sent using the PutS3MultipartUpload method. This multipart process saves state after each step so that
+        a large upload can be resumed with minimal loss if the processor or cluster is stopped and restarted.
+        A multipart upload consists of three steps:
+    </p>
+
+    <ol>
+        <li>Initiate upload</li>
+        <li>Upload the parts</li>
+        <li>Complete the upload</li>
+    </ol>
+
+    <p>
+        For multipart uploads, the processor saves state locally tracking the upload ID and parts uploaded, which must both be provided to complete the upload.
+        The AWS libraries select an endpoint URL based on the AWS region, but this can be overridden with the 'Endpoint Override URL' property for use with other
+        S3-compatible endpoints. The S3 API specifies that the maximum file size for a PutS3Object upload is 5GB. It also requires that parts in a multipart upload
+        must be at least 5MB in size, except for the last part. These limits establish the bounds for the Multipart Upload Threshold and Part Size properties.
+    </p>
 	<h2>Configuration Details</h2>
 	<h3>Object Key</h3>
 	<p>The Object Key property value should not start with "/".</p>
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAWSCredentials.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAWSCredentials.java
deleted file mode 100644
index 1009034dca..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAWSCredentials.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.aws;
-
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AnonymousAWSCredentials;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.internal.StaticCredentialsProvider;
-import com.amazonaws.services.s3.AmazonS3Client;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-/**
- * Unit tests for AWS Credential specification based on {@link AbstractAWSProcessor} and
- * [@link AbstractAWSCredentialsProviderProcessor},  without interaction with S3.
- */
-public class TestAWSCredentials {
-
-    private TestRunner runner = null;
-    private AbstractAWSProcessor mockAwsProcessor = null;
-    private AWSCredentials awsCredentials = null;
-    private AWSCredentialsProvider awsCredentialsProvider = null;
-
-    @BeforeEach
-    public void setUp() {
-        mockAwsProcessor = new AbstractAWSCredentialsProviderProcessor<AmazonS3Client>() {
-
-            protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-                return  Arrays.asList(
-                        AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE,
-                        AbstractAWSProcessor.CREDENTIALS_FILE,
-                        AbstractAWSProcessor.ACCESS_KEY,
-                        AbstractAWSProcessor.SECRET_KEY,
-                        AbstractAWSProcessor.TIMEOUT
-                );
-            }
-
-            @Override
-            protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
-                awsCredentials = credentials;
-                final AmazonS3Client s3 = new AmazonS3Client(credentials, config);
-                return s3;
-            }
-
-            @Override
-            protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
-                awsCredentialsProvider = credentialsProvider;
-                final AmazonS3Client s3 = new AmazonS3Client(credentialsProvider, config);
-                return s3;
-            }
-
-            @Override
-            public void onTrigger(final ProcessContext context, final ProcessSession session) {
-            }
-        };
-        runner = TestRunners.newTestRunner(mockAwsProcessor);
-    }
-
-    @Test
-    public void testAnonymousByDefault() {
-        runner.assertValid();
-        runner.run(1);
-
-        assertEquals(AnonymousAWSCredentials.class, awsCredentials.getClass());
-        assertNull(awsCredentialsProvider);
-    }
-
-    @Test
-    public void testAccessKeySecretKey() {
-        runner.setProperty(AbstractAWSProcessor.ACCESS_KEY, "testAccessKey");
-        runner.setProperty(AbstractAWSProcessor.SECRET_KEY, "testSecretKey");
-
-        runner.assertValid();
-        runner.run(1);
-
-        assertEquals(BasicAWSCredentials.class, awsCredentials.getClass());
-        assertNull(awsCredentialsProvider);
-    }
-
-    @Test
-    public void testCredentialsFile() {
-        runner.setProperty(AbstractAWSProcessor.CREDENTIALS_FILE, "src/test/resources/mock-aws-credentials.properties");
-
-        runner.assertValid();
-        runner.run(1);
-
-        assertEquals(PropertiesCredentials.class, awsCredentials.getClass());
-        assertNull(awsCredentialsProvider);
-    }
-
-
-    @Test
-    public void testCredentialsProviderControllerService() throws InitializationException {
-        final AWSCredentialsProviderControllerService credsService = new AWSCredentialsProviderControllerService();
-        runner.addControllerService("awsCredentialsProvider", credsService);
-        runner.setProperty(credsService, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(credsService, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
-        runner.enableControllerService(credsService);
-
-        runner.setProperty(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
-
-        runner.assertValid();
-        runner.run(1);
-
-        assertEquals(StaticCredentialsProvider.class, awsCredentialsProvider.getClass());
-        assertNull(awsCredentials);
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientCache.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientCache.java
deleted file mode 100644
index 1ad9c78040..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientCache.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.aws;
-
-import static com.amazonaws.regions.Regions.US_EAST_1;
-import static com.amazonaws.regions.Regions.US_WEST_2;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertSame;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.amazonaws.AmazonWebServiceClient;
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import org.apache.nifi.processor.ProcessContext;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.mockito.Mock;
-import org.mockito.junit.jupiter.MockitoExtension;
-
-@ExtendWith(MockitoExtension.class)
-public class TestAwsClientCache {
-
-    @Mock
-    private ProcessContext contextMock;
-    @Mock
-    private AwsClientProvider<AmazonWebServiceClient> awsClientProviderMock;
-    @Mock
-    private AmazonWebServiceClient awsClientMock1;
-    @Mock
-    private AmazonWebServiceClient awsClientMock2;
-    private AwsClientCache<AmazonWebServiceClient> clientCache;
-
-    @BeforeEach
-    public void setup() {
-        clientCache = new AwsClientCache<>();
-    }
-
-    @Test
-    public void testSameRegionUseExistingClientFromCache() {
-        final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
-        when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
-        final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
-
-        final AwsClientDetails newClientDetails = getClientDetails(US_WEST_2);
-        final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
-        verify(awsClientProviderMock, times(1)).createClient(eq(contextMock), any(AwsClientDetails.class));
-        assertSame(client1, client2);
-    }
-
-    @Test
-    public void testRegionChangeNewClientIsCreated() {
-        final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
-        when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
-        final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
-
-        final AwsClientDetails newClientDetails = getClientDetails(US_EAST_1);
-        when(awsClientProviderMock.createClient(contextMock, newClientDetails)).thenReturn(awsClientMock2);
-        final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
-        verify(awsClientProviderMock, times(2)).createClient(eq(contextMock), any(AwsClientDetails.class));
-        assertNotEquals(client1, client2);
-    }
-
-    @Test
-    public void testSameRegionClientCacheIsClearedNewClientIsCreated() {
-        final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
-        when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
-        final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
-
-        clientCache.clearCache();
-
-        final AwsClientDetails newClientDetails = getClientDetails(US_WEST_2);
-        when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock2);
-        final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
-        verify(awsClientProviderMock, times(2)).createClient(eq(contextMock), any(AwsClientDetails.class));
-        assertNotEquals(client1, client2);
-    }
-
-    private static AwsClientDetails getClientDetails(Regions region) {
-        return new AwsClientDetails(Region.getRegion(region));
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientDetails.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientDetails.java
deleted file mode 100644
index 6ea4257a29..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/TestAwsClientDetails.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.aws;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import org.junit.jupiter.api.Test;
-
-public class TestAwsClientDetails {
-
-    @Test
-    public void clientDetailsEqual() {
-        final AwsClientDetails details1 = getDefaultClientDetails(Regions.US_WEST_2);
-        final AwsClientDetails details2 = getDefaultClientDetails(Regions.US_WEST_2);
-
-        assertEquals(details1, details2);
-        assertEquals(details1.hashCode(), details2.hashCode());
-    }
-
-    @Test
-    public void clientDetailsDifferInRegion() {
-        final AwsClientDetails details1 = getDefaultClientDetails(Regions.US_WEST_2);
-        final AwsClientDetails details2 = getDefaultClientDetails(Regions.US_EAST_1);
-
-        assertNotEquals(details1, details2);
-        assertNotEquals(details1.hashCode(), details2.hashCode());
-    }
-
-    private AwsClientDetails getDefaultClientDetails(Regions region) {
-        return new AwsClientDetails(Region.getRegion(region));
-    }
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/MockAWSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/MockAWSProcessor.java
index de3c6c8c6e..dd61e337ad 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/MockAWSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/MockAWSProcessor.java
@@ -17,8 +17,9 @@
 package org.apache.nifi.processors.aws.credentials.provider.factory;
 
 import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.s3.AmazonS3Client;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -38,8 +39,8 @@ import static org.apache.nifi.processors.aws.credentials.provider.factory.Creden
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_PORT;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_CUSTOM_SIGNER_CLASS_NAME;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_CUSTOM_SIGNER_MODULE_LOCATION;
-import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_REGION;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_ENDPOINT;
+import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_REGION;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_SIGNER_OVERRIDE;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.MAX_SESSION_TIME;
 import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.PROFILE_NAME;
@@ -89,24 +90,9 @@ public class MockAWSProcessor extends AbstractAWSCredentialsProviderProcessor<Am
         return validationFailureResults;
     }
 
-    /**
-     * Create client using credentials provider. This is the preferred way for creating clients
-     */
-    @Override
-    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
-        getLogger().info("Creating client with credentials provider");
-        return new AmazonS3Client(credentialsProvider, config);
-    }
-
-    /**
-     * Create client using AWSCredentials
-     *
-     * @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
-     */
     @Override
-    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
-        getLogger().info("Creating client with awd credentials");
-        return new AmazonS3Client(credentials, config);
+    protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                          final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
+        return null;
     }
-
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/TestCredentialsProviderFactory.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/TestCredentialsProviderFactory.java
index 7d9264776a..6d2290a0aa 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/TestCredentialsProviderFactory.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/factory/TestCredentialsProviderFactory.java
@@ -26,7 +26,6 @@ import com.amazonaws.auth.PropertiesFileCredentialsProvider;
 import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
 import com.amazonaws.auth.Signer;
 import com.amazonaws.auth.profile.ProfileCredentialsProvider;
-import com.amazonaws.internal.StaticCredentialsProvider;
 import org.apache.nifi.processors.aws.credentials.provider.PropertiesCredentialsProvider;
 import org.apache.nifi.processors.aws.s3.FetchS3Object;
 import org.apache.nifi.processors.aws.signer.AwsSignerType;
@@ -89,7 +88,7 @@ public class TestCredentialsProviderFactory {
     public void testExplicitDefaultCredentialsExclusive() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
         runner.setProperty(CredentialPropertyDescriptors.USE_DEFAULT_CREDENTIALS, "true");
-        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
+        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
         runner.assertNotValid();
     }
 
@@ -97,7 +96,7 @@ public class TestCredentialsProviderFactory {
     public void testAccessKeyPairCredentials() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
         runner.setProperty(CredentialPropertyDescriptors.USE_DEFAULT_CREDENTIALS, "false");
-        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
+        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
         runner.setProperty(CredentialPropertyDescriptors.SECRET_KEY, "BogusSecretKey");
         runner.assertValid();
 
@@ -105,8 +104,6 @@ public class TestCredentialsProviderFactory {
         final CredentialsProviderFactory factory = new CredentialsProviderFactory();
         final AWSCredentialsProvider credentialsProvider = factory.getCredentialsProvider(runner.getProcessContext());
         assertNotNull(credentialsProvider);
-        assertEquals(StaticCredentialsProvider.class,
-                credentialsProvider.getClass(), "credentials provider should be equal");
 
         final AwsCredentialsProvider credentialsProviderV2 = factory.getAwsCredentialsProvider(runner.getProcessContext());
         assertNotNull(credentialsProviderV2);
@@ -117,14 +114,14 @@ public class TestCredentialsProviderFactory {
     @Test
     public void testAccessKeyPairIncomplete() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
-        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
+        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
         runner.assertNotValid();
     }
 
     @Test
     public void testAccessKeyPairIncompleteS3() throws Throwable {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
-        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
+        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
         runner.assertNotValid();
     }
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerServiceTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerServiceTest.java
index 3730e68a25..bc29bce1fb 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerServiceTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSCredentialsProviderControllerServiceTest.java
@@ -20,8 +20,6 @@ import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
 import com.amazonaws.auth.PropertiesFileCredentialsProvider;
 import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
-import com.amazonaws.internal.StaticCredentialsProvider;
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
 import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
 import org.apache.nifi.processors.aws.s3.FetchS3Object;
 import org.apache.nifi.util.TestRunner;
@@ -59,8 +57,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.enableControllerService(serviceImpl);
 
         runner.assertValid(serviceImpl);
@@ -69,8 +67,6 @@ public class AWSCredentialsProviderControllerServiceTest {
         assertNotNull(service);
         final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
         assertNotNull(credentialsProvider);
-        assertEquals(StaticCredentialsProvider.class,
-                credentialsProvider.getClass(), "credentials provider should be equal");
     }
 
     @Test
@@ -78,8 +74,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@@ -100,8 +96,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@@ -123,8 +119,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@@ -139,8 +135,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@@ -155,8 +151,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
@@ -168,8 +164,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
@@ -181,8 +177,8 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
 
         runner.assertNotValid(serviceImpl);
@@ -193,7 +189,7 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
                 "src/test/resources/mock-aws-credentials.properties");
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
         runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
@@ -247,7 +243,7 @@ public class AWSCredentialsProviderControllerServiceTest {
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
         runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
                 "src/test/resources/mock-aws-credentials.properties");
-        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
         runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
 
         runner.assertNotValid(serviceImpl);
@@ -260,7 +256,7 @@ public class AWSCredentialsProviderControllerServiceTest {
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
         runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
                 "src/test/resources/mock-aws-credentials.properties");
-        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
 
         runner.assertNotValid(serviceImpl);
     }
@@ -282,7 +278,7 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
 
         runner.assertNotValid(serviceImpl);
     }
@@ -302,7 +298,7 @@ public class AWSCredentialsProviderControllerServiceTest {
         final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
         final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
         runner.addControllerService("awsCredentialsProvider", serviceImpl);
-        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "${literal(\"awsAccessKey\")}");
+        runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "${literal(\"awsAccessKey\")}");
         runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "${literal(\"awsSecretKey\")}");
         runner.enableControllerService(serviceImpl);
 
@@ -313,7 +309,7 @@ public class AWSCredentialsProviderControllerServiceTest {
 
         assertEquals(
                 "awsAccessKey", service.getCredentialsProvider().getCredentials().getAWSAccessKeyId(),
-                "Expression language should be supported for " + CredentialPropertyDescriptors.ACCESS_KEY.getName());
+                "Expression language should be supported for " + CredentialPropertyDescriptors.ACCESS_KEY_ID.getName());
         assertEquals(
                 "awsSecretKey", service.getCredentialsProvider().getCredentials().getAWSSecretKey(),
                 "Expression language should be supported for " + CredentialPropertyDescriptors.SECRET_KEY.getName());
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSProcessorProxyTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSProcessorProxyTest.java
index c587e3d1ea..8f9d9c07cd 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSProcessorProxyTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/credentials/provider/service/AWSProcessorProxyTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.processors.aws.credentials.provider.service;
 
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
 import org.apache.nifi.processors.aws.s3.FetchS3Object;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -29,79 +29,68 @@ public class AWSProcessorProxyTest {
     private TestRunner runner;
 
     @BeforeEach
-    public void testSetup() throws Throwable {
+    public void testSetup() {
         runner = TestRunners.newTestRunner(FetchS3Object.class);
-        runner.setProperty(FetchS3Object.BUCKET, "bucket");
+        runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "bucket");
         runner.assertValid();
     }
 
     @AfterEach
-    public void testTearDown() throws Throwable {
+    public void testTearDown() {
         runner = null;
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyHostOnlyInvalid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
+    public void testProxyHostOnlyInvalid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
         runner.assertNotValid();
     }
-    @SuppressWarnings("deprecation")
+
     @Test
-    public void testProxyHostPortOnlyInvalid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
+    public void testProxyHostPortOnlyInvalid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
         runner.assertNotValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyHostPortNonNumberInvalid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "a");
+    public void testProxyHostPortNonNumberInvalid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "a");
         runner.assertNotValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyHostAndPortValid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
+    public void testProxyHostAndPortValid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
         runner.assertValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyUserNoPasswordInValid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
+    public void testProxyUserNoPasswordInValid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
         runner.assertNotValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyNoUserPasswordInValid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
+    public void testProxyNoUserPasswordInValid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
         runner.assertNotValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyUserPasswordNoHostInValid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
-        runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
+    public void testProxyUserPasswordNoHostInValid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
         runner.assertNotValid();
     }
 
-    @SuppressWarnings("deprecation")
     @Test
-    public void testProxyUserPasswordHostValid() throws Throwable {
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
-        runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
-        runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
-        runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
+    public void testProxyUserPasswordHostValid() {
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
+        runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
         runner.assertValid();
     }
 
-
-
-
-
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBTest.java
index 486bc31221..91da2f8c53 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBTest.java
@@ -18,6 +18,9 @@ package org.apache.nifi.processors.aws.dynamodb;
 
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.AmazonServiceException.ErrorType;
+import org.apache.nifi.util.MockFlowFile;
+
+import java.util.List;
 
 /**
  * Provides reused elements and utilities for the AWS DynamoDB related tests
@@ -27,6 +30,21 @@ import com.amazonaws.AmazonServiceException.ErrorType;
  * @see DeleteDynamoDBTest
  */
 public abstract class AbstractDynamoDBTest {
+    public static final String REGION = "us-west-2";
+    public static final String stringHashStringRangeTableName = "StringHashStringRangeTable";
+
+    private static final List<String> errorAttributes = List.of(
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_CODE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_MESSAGE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_TYPE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_SERVICE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_REQUEST_ID,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_STATUS_CODE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE,
+            AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE
+    );
 
     protected AmazonServiceException getSampleAwsServiceException() {
         final AmazonServiceException testServiceException = new AmazonServiceException("Test AWS Service Exception");
@@ -38,4 +56,9 @@ public abstract class AbstractDynamoDBTest {
 
         return testServiceException;
     }
+
+    protected static void validateServiceExceptionAttributes(final MockFlowFile flowFile) {
+        errorAttributes.forEach(flowFile::assertAttributeExists);
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDBTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDBTest.java
index 89f768d195..2d58243bb1 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDBTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/DeleteDynamoDBTest.java
@@ -40,8 +40,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
@@ -101,7 +99,7 @@ public class DeleteDynamoDBTest extends AbstractDynamoDBTest {
 
         List<MockFlowFile> flowFiles = deleteRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         for (MockFlowFile flowFile : flowFiles) {
-            ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
+            validateServiceExceptionAttributes(flowFile);
         }
 
     }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/GetDynamoDBTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/GetDynamoDBTest.java
index a1c5f70f15..f4233d5f6b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/GetDynamoDBTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/GetDynamoDBTest.java
@@ -39,7 +39,6 @@ import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.util.ArrayList;
-
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -47,8 +46,6 @@ import java.util.Map;
 
 import static org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
 import static org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -411,7 +408,7 @@ public class GetDynamoDBTest extends AbstractDynamoDBTest {
 
         List<MockFlowFile> flowFiles = getRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         for (MockFlowFile flowFile : flowFiles) {
-            ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
+            validateServiceExceptionAttributes(flowFile);
         }
 
     }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/ITAbstractDynamoDBTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/ITAbstractDynamoDBTest.java
deleted file mode 100644
index 4f835d04f1..0000000000
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/ITAbstractDynamoDBTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.aws.dynamodb;
-
-import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.document.Table;
-import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
-import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
-import com.amazonaws.services.dynamodbv2.model.DeleteTableResult;
-import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
-import com.amazonaws.services.dynamodbv2.model.KeyType;
-import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
-import org.apache.nifi.flowfile.FlowFile;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-
-import java.io.FileInputStream;
-import java.util.ArrayList;
-
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-
-public class ITAbstractDynamoDBTest {
-
-    protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
-    protected static DynamoDB dynamoDB;
-    protected static AmazonDynamoDBClient amazonDynamoDBClient;
-    protected static String stringHashStringRangeTableName = "StringHashStringRangeTable";
-    protected static String numberHashNumberRangeTableName = "NumberHashNumberRangeTable";
-    protected static String numberHashOnlyTableName = "NumberHashOnlyTable";
-    protected final static String REGION = "us-west-2";
-
-    @BeforeAll
-    public static void beforeClass() throws Exception {
-        FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
-        final PropertiesCredentials credentials = new PropertiesCredentials(fis);
-        amazonDynamoDBClient = new AmazonDynamoDBClient(credentials);
-        dynamoDB = new DynamoDB(amazonDynamoDBClient);
-        amazonDynamoDBClient.setRegion(Region.getRegion(Regions.US_WEST_2));
-
-        ArrayList<AttributeDefinition> attributeDefinitions = new ArrayList<AttributeDefinition>();
-        attributeDefinitions
-            .add(new AttributeDefinition().withAttributeName("hashS").withAttributeType("S"));
-        attributeDefinitions
-            .add(new AttributeDefinition().withAttributeName("rangeS").withAttributeType("S"));
-
-        ArrayList<KeySchemaElement> keySchema = new ArrayList<KeySchemaElement>();
-        keySchema.add(new KeySchemaElement().withAttributeName("hashS").withKeyType(KeyType.HASH));
-        keySchema.add(new KeySchemaElement().withAttributeName("rangeS").withKeyType(KeyType.RANGE));
-
-        CreateTableRequest request = new CreateTableRequest()
-            .withTableName(stringHashStringRangeTableName)
-            .withKeySchema(keySchema)
-            .withAttributeDefinitions(attributeDefinitions)
-            .withProvisionedThroughput(new ProvisionedThroughput()
-                .withReadCapacityUnits(5L)
-                .withWriteCapacityUnits(6L));
-        Table stringHashStringRangeTable = dynamoDB.createTable(request);
-        stringHashStringRangeTable.waitForActive();
-
-        attributeDefinitions = new ArrayList<AttributeDefinition>();
-        attributeDefinitions
-            .add(new AttributeDefinition().withAttributeName("hashN").withAttributeType("N"));
-        attributeDefinitions
-            .add(new AttributeDefinition().withAttributeName("rangeN").withAttributeType("N"));
-
-        keySchema = new ArrayList<KeySchemaElement>();
-        keySchema.add(new KeySchemaElement().withAttributeName("hashN").withKeyType(KeyType.HASH));
-        keySchema.add(new KeySchemaElement().withAttributeName("rangeN").withKeyType(KeyType.RANGE));
-
-        request = new CreateTableRequest()
-            .withTableName(numberHashNumberRangeTableName)
-            .withKeySchema(keySchema)
-            .withAttributeDefinitions(attributeDefinitions)
-            .withProvisionedThroughput(new ProvisionedThroughput()
-                .withReadCapacityUnits(5L)
-                .withWriteCapacityUnits(6L));
-        Table numberHashNumberRangeTable = dynamoDB.createTable(request);
-        numberHashNumberRangeTable.waitForActive();
-
-        attributeDefinitions = new ArrayList<AttributeDefinition>();
-        attributeDefinitions
-            .add(new AttributeDefinition().withAttributeName("hashN").withAttributeType("N"));
-
-        keySchema = new ArrayList<KeySchemaElement>();
-        keySchema.add(new KeySchemaElement().withAttributeName("hashN").withKeyType(KeyType.HASH));
-
-        request = new CreateTableRequest()
-            .withTableName(numberHashOnlyTableName)
-            .withKeySchema(keySchema)
-            .withAttributeDefinitions(attributeDefinitions)
-            .withProvisionedThroughput(new ProvisionedThroughput()
-                .withReadCapacityUnits(5L)
-                .withWriteCapacityUnits(6L));
-        Table numberHashOnlyTable = dynamoDB.createTable(request);
-        numberHashOnlyTable.waitForActive();
-
-    }
-
-    protected static void validateServiceExceptionAttribute(FlowFile flowFile) {
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_CODE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_MESSAGE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_TYPE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_SERVICE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_REQUEST_ID));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_STATUS_CODE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE));
-        assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE));
-    }
-
-    @AfterAll
-    public static void afterClass() {
-        DeleteTableResult result = amazonDynamoDBClient.deleteTable(stringHashStringRangeTableName);
-        result = amazonDynamoDBClient.deleteTable(numberHashNumberRangeTableName);
-        result = amazonDynamoDBClient.deleteTable(numberHashOnlyTableName);
-    }
-
-
-}
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDBTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDBTest.java
index a695f34632..38a2b579b3 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDBTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/dynamodb/PutDynamoDBTest.java
@@ -36,12 +36,11 @@ import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
-import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
@@ -54,7 +53,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
     @BeforeEach
     public void setUp() {
         outcome = new BatchWriteItemOutcome(result);
-        result.setUnprocessedItems(new HashMap<String, List<WriteRequest>>());
+        result.setUnprocessedItems(new HashMap<>());
         final DynamoDB mockDynamoDB = new DynamoDB(Regions.AP_NORTHEAST_1) {
             @Override
             public BatchWriteItemOutcome batchWriteItem(TableWriteItems... tableWriteItems) {
@@ -68,7 +67,6 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
                 return mockDynamoDB;
             }
         };
-
     }
 
     @Test
@@ -103,7 +101,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
 
         List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         for (MockFlowFile flowFile : flowFiles) {
-            ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
+            validateServiceExceptionAttributes(flowFile);
         }
 
     }
@@ -195,15 +193,15 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
         putRunner.run(1);
 
-        putRunner.assertAllFlowFilesTransferred(AbstractWriteDynamoDBProcessor.REL_SUCCESS, 1);
+        putRunner.assertAllFlowFilesTransferred(AbstractDynamoDBProcessor.REL_SUCCESS, 1);
 
-        List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
+        List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
         for (MockFlowFile flowFile : flowFiles) {
             System.out.println(flowFile.getAttributes());
             assertEquals(document, new String(flowFile.toByteArray()));
@@ -223,27 +221,25 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
         byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
-        for (int i = 0; i < item.length; i++) {
-            item[i] = 'a';
-        }
+        Arrays.fill(item, (byte) 'a');
         String document2 = new String(item);
         putRunner.enqueue(document2.getBytes());
 
         putRunner.run(2,true,true);
 
-        List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
+        List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         for (MockFlowFile flowFile : flowFilesFailed) {
             System.out.println(flowFile.getAttributes());
             flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
             assertEquals(item.length,flowFile.getSize());
         }
 
-        List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
+        List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
         for (MockFlowFile flowFile : flowFilesSuccessful) {
             System.out.println(flowFile.getAttributes());
             assertEquals(document, new String(flowFile.toByteArray()));
@@ -262,27 +258,25 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
         byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
-        for (int i = 0; i < item.length; i++) {
-            item[i] = 'a';
-        }
+        Arrays.fill(item, (byte) 'a');
         String document2 = new String(item);
         putRunner.enqueue(document2.getBytes());
 
         putRunner.run(1);
 
-        List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
+        List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         for (MockFlowFile flowFile : flowFilesFailed) {
             System.out.println(flowFile.getAttributes());
             flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
             assertEquals(item.length,flowFile.getSize());
         }
 
-        List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
+        List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
         for (MockFlowFile flowFile : flowFilesSuccessful) {
             System.out.println(flowFile.getAttributes());
             assertEquals(document, new String(flowFile.toByteArray()));
@@ -301,26 +295,23 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
-        for (int i = 0; i < item.length; i++) {
-            item[i] = 'a';
-        }
+        Arrays.fill(item, (byte) 'a');
         String document = new String(item);
         putRunner.enqueue(document.getBytes());
 
         putRunner.run(1);
 
-        putRunner.assertAllFlowFilesTransferred(AbstractWriteDynamoDBProcessor.REL_FAILURE, 1);
+        putRunner.assertAllFlowFilesTransferred(AbstractDynamoDBProcessor.REL_FAILURE, 1);
 
-        List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
+        List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
         assertEquals(1,flowFiles.size());
         for (MockFlowFile flowFile : flowFiles) {
             System.out.println(flowFile.getAttributes());
             flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
             assertEquals(item.length,flowFile.getSize());
         }
-
     }
 
     @Test
@@ -348,7 +339,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
@@ -388,7 +379,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
@@ -427,7 +418,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
         putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
         putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
-        putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
+        putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
         String document = "{\"name\":\"john\"}";
         putRunner.enqueue(document.getBytes());
 
@@ -443,9 +434,8 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
 
     @Test
     public void testStringHashStringRangePutSuccessfulWithMockOneUnprocessed() {
-        Map<String, List<WriteRequest>> unprocessed =
-                new HashMap<String, List<WriteRequest>>();
-        PutRequest put = new PutRequest();
+        final Map<String, List<WriteRequest>> unprocessed = new HashMap<>();
+        final PutRequest put = new PutRequest();
         put.addItemEntry("hashS", new AttributeValue("h1"));
         put.addItemEntry("rangeS", new AttributeValue("r1"));
         WriteRequest write = new WriteRequest(put);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/ITPutKinesisFirehose.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/ITPutKinesisFirehose.java
index eb3e317520..d8288699b6 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/ITPutKinesisFirehose.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/ITPutKinesisFirehose.java
@@ -20,10 +20,12 @@ import org.apache.nifi.processors.aws.s3.FetchS3Object;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
+import java.io.File;
 import java.util.List;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -32,44 +34,38 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 /**
  * This test contains both unit and integration test (integration tests are ignored by default)
  */
-public class ITPutKinesisFirehose {
 
+public class ITPutKinesisFirehose {
+    private final byte[] ONE_MB = new byte[1000 * 1024];
     private TestRunner runner;
     protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
 
+    @BeforeAll
+    public static void testSkipping() {
+        Assumptions.assumeTrue(new File(CREDENTIALS_FILE).exists());
+    }
+
     @BeforeEach
     public void setUp() throws Exception {
         runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.ACCESS_KEY, "abcd");
-        runner.setProperty(PutKinesisFirehose.SECRET_KEY, "secret key");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "deliveryName");
-        runner.assertValid();
+        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
+        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
     }
 
-    @AfterEach
-    public void tearDown() throws Exception {
-        runner = null;
-    }
 
     /**
      * Comment out ignore for integration tests (requires creds files)
      */
     @Test
     public void testIntegrationSuccess() throws Exception {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-
         runner.enqueue("test".getBytes());
-        runner.run(1);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 1);
 
         final List<MockFlowFile> ffs = runner.getFlowFilesForRelationship(FetchS3Object.REL_SUCCESS);
         final MockFlowFile out = ffs.iterator().next();
-
-        out.assertContentEquals("test".getBytes());
+        out.assertContentEquals("test");
     }
 
     /**
@@ -77,57 +73,33 @@ public class ITPutKinesisFirehose {
      */
     @Test
     public void testIntegrationFailedBadStreamName() throws Exception {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "bad-firehose-s3-test");
-        runner.assertValid();
-
-        runner.enqueue("test".getBytes());
-        runner.run(1);
 
+        runner.enqueue("test");
+        runner.run();
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_FAILURE, 1);
-
     }
 
     @Test
     public void testOneMessageWithMaxBufferSizeGreaterThan1MBOneSuccess() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
         runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.run(1);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
-        assertEquals(1,flowFiles.size());
     }
 
     @Test
     public void testTwoMessageBatch5WithMaxBufferSize1MBRunOnceTwoMessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "5");
         runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "2 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.run(1);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
-        assertEquals(2,flowFiles.size());
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         for (MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
@@ -135,73 +107,43 @@ public class ITPutKinesisFirehose {
 
     @Test
     public void testThreeMessageWithBatch10MaxBufferSize1MBTRunOnceTwoMessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes.clone());
-        runner.run(1);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
-        assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
     }
 
     @Test
     public void testTwoMessageWithBatch10MaxBufferSize1MBTRunOnceTwoMessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.run(1);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
-        assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
     }
 
     @Test
     public void testThreeMessageWithBatch2MaxBufferSize1MBRunTwiceThreeMessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes.clone());
-        runner.run(2, true, true);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run(2);
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 3);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
-        assertEquals(3,flowFiles.size());
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         for (MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
@@ -209,193 +151,134 @@ public class ITPutKinesisFirehose {
 
     @Test
     public void testThreeMessageHello2MBThereWithBatch10MaxBufferSize1MBRunOnceTwoMessageSuccessOneFailed() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
         runner.enqueue("hello".getBytes());
-        runner.enqueue(bytes);
+        runner.enqueue(ONE_MB);
         runner.enqueue("there".getBytes());
-        runner.run(1, true, true);
+        runner.run();
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
 
-        List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
+        final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
         assertEquals(1,flowFilesFailed.size());
-        for (MockFlowFile flowFileFailed : flowFilesFailed) {
-            assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
+        for (final MockFlowFile flowFileFailed : flowFilesFailed) {
+            flowFileFailed.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE);
         }
     }
 
     @Test
     public void testTwoMessageHello2MBWithBatch10MaxBufferSize1MBRunOnceOneSuccessOneFailed() throws Exception {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
         runner.enqueue("hello".getBytes());
-        runner.enqueue(bytes);
+        runner.enqueue(ONE_MB);
         runner.run(1, true, true);
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(1,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
             flowFile.assertContentEquals("hello".getBytes());
         }
 
-        List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
+        final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
         assertEquals(1,flowFilesFailed.size());
-        for (MockFlowFile flowFileFailed : flowFilesFailed) {
+        for (final MockFlowFile flowFileFailed : flowFilesFailed) {
             assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
         }
     }
 
     @Test
     public void testTwoMessage2MBHelloWorldWithBatch10MaxBufferSize1MBRunOnceTwoMessageSent() throws Exception {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
+        runner.enqueue(ONE_MB);
         runner.enqueue("HelloWorld".getBytes());
-        runner.run(1, true, true);
+        runner.run();
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(1,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
             flowFile.assertContentEquals("HelloWorld".getBytes());
         }
 
-        List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
+        final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
         assertEquals(1,flowFilesFailed.size());
-        for (MockFlowFile flowFileFailed : flowFilesFailed) {
+        for (final MockFlowFile flowFileFailed : flowFilesFailed) {
             assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
         }
     }
 
     @Test
     public void testTwoMessageHelloWorldWithBatch10MaxBufferSize1MBRunOnceTwoMessageSent() throws Exception {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
         runner.enqueue("Hello".getBytes());
         runner.enqueue("World".getBytes());
-        runner.run(1, true, true);
+        runner.run();
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
         flowFiles.get(0).assertContentEquals("Hello".getBytes());
         flowFiles.get(1).assertContentEquals("World".getBytes());
 
-        List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
+        final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
         assertEquals(0,flowFilesFailed.size());
     }
 
     @Test
     public void testThreeMessageWithBatch5MaxBufferSize1MBRunOnceTwoMessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "5");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes.clone());
-        runner.run(1, true, true);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
     }
 
     @Test
     public void test5MessageWithBatch10MaxBufferSize10MBRunOnce5MessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[10];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.run(1, true, true);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 5);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(5,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
     }
 
     @Test
     public void test5MessageWithBatch2MaxBufferSize10MBRunOnce2MessageSent() {
-        runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
-        runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
-        runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
-        runner.assertValid();
-        byte [] bytes = new byte[10];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes.clone());
-        runner.enqueue(bytes);
-        runner.enqueue(bytes.clone());
-        runner.run(1, true, true);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.enqueue(ONE_MB);
+        runner.run();
 
         runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
         assertEquals(2,flowFiles.size());
-        for (MockFlowFile flowFile : flowFiles) {
+        for (final MockFlowFile flowFile : flowFiles) {
             flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
         }
     }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/TestPutKinesisFirehose.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/TestPutKinesisFirehose.java
index fb2ed5178e..0e8a4b1d56 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/TestPutKinesisFirehose.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/firehose/TestPutKinesisFirehose.java
@@ -29,13 +29,10 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 public class TestPutKinesisFirehose {
     private TestRunner runner;
-    protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
 
     @BeforeEach
     public void setUp() throws Exception {
         runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
-        runner.setProperty(PutKinesisFirehose.ACCESS_KEY, "abcd");
-        runner.setProperty(PutKinesisFirehose.SECRET_KEY, "secret key");
         runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "deliveryName");
         runner.assertValid();
     }
@@ -66,10 +63,7 @@ public class TestPutKinesisFirehose {
     public void testWithSizeGreaterThan1MB() {
         runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "1");
         runner.assertValid();
-        byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE + 1)];
-        for (int i = 0; i < bytes.length; i++) {
-            bytes[i] = 'a';
-        }
+        byte[] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE + 1)];
         runner.enqueue(bytes);
         runner.run(1);
 
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
index 34b484a0fa..c6ac46cda4 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
@@ -74,7 +74,7 @@ public class TestConsumeKinesisStream {
     public void testValidWithCredentials() throws InitializationException {
         final ControllerService credentialsProvider = new AWSCredentialsProviderControllerService();
         runner.addControllerService("credentials-provider", credentialsProvider);
-        runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.ACCESS_KEY, "access-key");
+        runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.ACCESS_KEY_ID, "access-key");
         runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.SECRET_KEY, "secret-key");
         runner.assertValid(credentialsProvider);
         runner.enableControllerService(credentialsProvider);
@@ -232,10 +232,6 @@ public class TestConsumeKinesisStream {
         runWorker(true, true);
     }
 
-    @Test
-    public void testRunWorkerWithoutCredentials() throws UnknownHostException, InitializationException, InterruptedException {
-        runWorker(false, false);
-    }
 
     @Test
     public void testInvalidDynamicKCLProperties() {
@@ -341,6 +337,7 @@ public class TestConsumeKinesisStream {
     private void runWorker(final boolean withCredentials, final boolean waitForFailure) throws UnknownHostException, InitializationException, InterruptedException {
         final TestRunner mockConsumeKinesisStreamRunner = TestRunners.newTestRunner(MockConsumeKinesisStream.class);
 
+        mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.GRACEFUL_SHUTDOWN_TIMEOUT, "50 millis");
         mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.KINESIS_STREAM_NAME, "test-stream");
         mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.APPLICATION_NAME, "test-application");
         mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.REGION, Regions.EU_WEST_2.getName());
@@ -349,7 +346,7 @@ public class TestConsumeKinesisStream {
         final AWSCredentialsProviderService awsCredentialsProviderService = new AWSCredentialsProviderControllerService();
         mockConsumeKinesisStreamRunner.addControllerService("aws-credentials", awsCredentialsProviderService);
         if (withCredentials) {
-            mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.ACCESS_KEY, "test-access");
+            mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.ACCESS_KEY_ID, "test-access");
             mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.SECRET_KEY, "test-secret");
         } else {
             mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.USE_ANONYMOUS_CREDENTIALS, "true");
@@ -380,16 +377,12 @@ public class TestConsumeKinesisStream {
 
         if (!waitForFailure) {
             // re-trigger the processor to ensure the Worker isn't re-initialised when already running
-            mockConsumeKinesisStreamRunner.run(1, false, false);
+            mockConsumeKinesisStreamRunner.run(1, true, false);
             assertTrue(((MockProcessContext) mockConsumeKinesisStreamRunner.getProcessContext()).isYieldCalled());
-
-            // stop the processor
-            mockConsumeKinesisStreamRunner.stop();
         } else {
             for (int runs = 0; runs < 10; runs++) {
                 try {
                     mockConsumeKinesisStreamRunner.run(1, false, false);
-                    Thread.sleep(1_000);
                 } catch (AssertionError e) {
                     assertThat(e.getCause(), instanceOf(ProcessException.class));
                     assertThat(e.getCause().getMessage(), equalTo("Worker has shutdown unexpectedly, possibly due to a configuration issue; check logs for details"));
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java
index 18393e7b58..d9b7b51ffc 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java
@@ -17,24 +17,15 @@
 
 package org.apache.nifi.processors.aws.ml.polly;
 
-import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_ORIGINAL;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.Mockito.when;
-
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.polly.AmazonPollyClient;
 import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
 import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
 import com.amazonaws.services.polly.model.SynthesisTask;
 import com.amazonaws.services.polly.model.TaskStatus;
-import java.util.Collections;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
 import org.apache.nifi.reporting.InitializationException;
@@ -48,6 +39,18 @@ import org.mockito.Captor;
 import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
+import java.util.Collections;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
 @ExtendWith(MockitoExtension.class)
 public class GetAwsPollyStatusTest {
     private static final String TEST_TASK_ID = "testTaskId";
@@ -66,7 +69,8 @@ public class GetAwsPollyStatusTest {
 
         final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
             @Override
-            protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+            protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                     final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
                 return mockPollyClient;
             }
         };
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatusTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatusTest.java
index 948d9783eb..82fb01cdcf 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatusTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatusTest.java
@@ -17,17 +17,10 @@
 
 package org.apache.nifi.processors.aws.ml.textract;
 
-import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
-import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TEXTRACT_TYPE;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.Mockito.when;
-
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.textract.AmazonTextractClient;
 import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
 import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
@@ -46,6 +39,15 @@ import org.mockito.Captor;
 import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TEXTRACT_TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
 @ExtendWith(MockitoExtension.class)
 public class GetAwsTextractJobStatusTest {
     private static final String TEST_TASK_ID = "testTaskId";
@@ -62,7 +64,8 @@ public class GetAwsTextractJobStatusTest {
         when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredentialProvider");
         final GetAwsTextractJobStatus awsTextractJobStatusGetter = new GetAwsTextractJobStatus() {
             @Override
-            protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+            protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                        final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
                 return mockTextractClient;
             }
         };
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java
index 89a94aeb8e..68a253dc7a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java
@@ -17,24 +17,16 @@
 
 package org.apache.nifi.processors.aws.ml.transcribe;
 
-import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.Mockito.when;
-
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.transcribe.AmazonTranscribeClient;
 import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
 import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
 import com.amazonaws.services.transcribe.model.Transcript;
 import com.amazonaws.services.transcribe.model.TranscriptionJob;
 import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
-import java.util.Collections;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
 import org.apache.nifi.reporting.InitializationException;
@@ -48,6 +40,17 @@ import org.mockito.Captor;
 import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
+import java.util.Collections;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
 @ExtendWith(MockitoExtension.class)
 public class GetAwsTranscribeJobStatusTest {
     private static final String TEST_TASK_ID = "testTaskId";
@@ -68,7 +71,8 @@ public class GetAwsTranscribeJobStatusTest {
         when(mockAwsCredentialsProvider.getIdentifier()).thenReturn(AWS_CREDENTIAL_PROVIDER_NAME);
         final GetAwsTranscribeJobStatus mockPollyFetcher = new GetAwsTranscribeJobStatus() {
             @Override
-            protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentials, ClientConfiguration config) {
+            protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                          final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
                 return mockTranscribeClient;
             }
         };
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java
index 1475c99c55..7c868c70ce 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java
@@ -17,24 +17,16 @@
 
 package org.apache.nifi.processors.aws.ml.translate;
 
-import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
-import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.Mockito.when;
-
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
 import com.amazonaws.services.translate.AmazonTranslateClient;
 import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
 import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
 import com.amazonaws.services.translate.model.JobStatus;
 import com.amazonaws.services.translate.model.OutputDataConfig;
 import com.amazonaws.services.translate.model.TextTranslationJobProperties;
-import java.util.Collections;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
 import org.apache.nifi.reporting.InitializationException;
@@ -48,6 +40,17 @@ import org.mockito.Captor;
 import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
 
+import java.util.Collections;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
 @ExtendWith(MockitoExtension.class)
 public class GetAwsTranslateJobStatusTest {
     private static final String TEST_TASK_ID = "testTaskId";
@@ -68,7 +71,8 @@ public class GetAwsTranslateJobStatusTest {
         when(mockAwsCredentialsProvider.getIdentifier()).thenReturn(AWS_CREDENTIALS_PROVIDER_NAME);
         final GetAwsTranslateJobStatus mockPollyFetcher = new GetAwsTranslateJobStatus() {
             @Override
-            protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+            protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
+                                                         final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
                 return mockTranslateClient;
             }
         };
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/AbstractS3IT.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/AbstractS3IT.java
index 6e40816837..aeec0f4830 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/AbstractS3IT.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/AbstractS3IT.java
@@ -16,17 +16,17 @@
  */
 package org.apache.nifi.processors.aws.s3;
 
-import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.regions.Region;
-import com.amazonaws.regions.Regions;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.client.builder.AwsClientBuilder;
 import com.amazonaws.services.kms.AWSKMS;
 import com.amazonaws.services.kms.AWSKMSClient;
 import com.amazonaws.services.kms.model.CreateKeyRequest;
 import com.amazonaws.services.kms.model.CreateKeyResult;
 import com.amazonaws.services.kms.model.GenerateDataKeyRequest;
 import com.amazonaws.services.kms.model.GenerateDataKeyResult;
-import com.amazonaws.services.kms.model.ScheduleKeyDeletionRequest;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CreateBucketRequest;
 import com.amazonaws.services.s3.model.DeleteBucketRequest;
@@ -34,23 +34,32 @@ import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.ObjectTagging;
 import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.amazonaws.services.s3.model.Tag;
-import org.apache.nifi.util.file.FileUtils;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.testcontainers.containers.localstack.LocalStackContainer;
+import org.testcontainers.utility.DockerImageName;
 
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -63,57 +72,70 @@ import static org.junit.jupiter.api.Assertions.fail;
  * @see ITListS3
  */
 public abstract class AbstractS3IT {
-    protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
     protected final static String SAMPLE_FILE_RESOURCE_NAME = "/hello.txt";
-    protected final static String REGION = System.getProperty("it.aws.region", "us-west-1");
-    // Adding REGION to bucket prevents errors of
-    //      "A conflicting conditional operation is currently in progress against this resource."
-    // when bucket is rapidly added/deleted and consistency propagation causes this error.
-    // (Should not be necessary if REGION remains static, but added to prevent future frustration.)
-    // [see http://stackoverflow.com/questions/13898057/aws-error-message-a-conflicting-conditional-operation-is-currently-in-progress]
-    protected final static String BUCKET_NAME = "test-bucket-" + System.currentTimeMillis() + "-" + REGION;
-
-    // Static so multiple Tests can use same client
-    protected static AmazonS3Client client;
-    protected static AWSKMS kmsClient;
+    protected final static String BUCKET_NAME = "test-bucket-" + System.currentTimeMillis();
+
+    private static AmazonS3 client;
+    private static AWSKMS kmsClient;
+    private final List<String> addedKeys = new ArrayList<>();
+
+    private static final DockerImageName localstackImage = DockerImageName.parse("localstack/localstack:latest");
+
+    private static final LocalStackContainer localstack = new LocalStackContainer(localstackImage)
+            .withServices(LocalStackContainer.Service.S3, LocalStackContainer.Service.KMS);
+
 
     @BeforeAll
     public static void oneTimeSetup() {
-        // Creates a client and bucket for this test
+        localstack.start();
 
-        final FileInputStream fis;
-        try {
-            fis = new FileInputStream(CREDENTIALS_FILE);
-        } catch (FileNotFoundException e1) {
-            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e1.getLocalizedMessage());
-            return;
-        }
-        try {
-            final PropertiesCredentials credentials = new PropertiesCredentials(fis);
-            client = new AmazonS3Client(credentials);
-            kmsClient = new AWSKMSClient(credentials);
-            kmsClient.setRegion(Region.getRegion(Regions.fromName(REGION)));
+        client = AmazonS3ClientBuilder.standard()
+                .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(localstack.getEndpoint().toString(), localstack.getRegion()))
+                .withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey())))
+                .build();
 
-            if (client.doesBucketExist(BUCKET_NAME)) {
-                fail("Bucket " + BUCKET_NAME + " exists. Choose a different bucket name to continue test");
-            }
+        kmsClient = AWSKMSClient.builder()
+                .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(localstack.getEndpoint().toString(), localstack.getRegion()))
+                .withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey())))
+                .build();
 
-            CreateBucketRequest request = REGION.contains("east")
-                    ? new CreateBucketRequest(BUCKET_NAME) // See https://github.com/boto/boto3/issues/125
-                    : new CreateBucketRequest(BUCKET_NAME, REGION);
-            client.createBucket(request);
+        final CreateBucketRequest request = new CreateBucketRequest(BUCKET_NAME);
+        client.createBucket(request);
+    }
 
-        } catch (final AmazonS3Exception e) {
-            fail("Can't create the key " + BUCKET_NAME + ": " + e.getLocalizedMessage());
-        } catch (final IOException e) {
-            fail("Caught IOException preparing tests: " + e.getLocalizedMessage());
-        } finally {
-            FileUtils.closeQuietly(fis);
-        }
+    protected AmazonS3 getClient() {
+        return client;
+    }
+
+    protected AWSKMS getKmsClient() {
+        return kmsClient;
+    }
+
+    protected String getEndpointOverride() {
+        return localstack.getEndpointOverride(LocalStackContainer.Service.S3).toString();
+    }
+
+    protected static String getRegion() {
+        return localstack.getRegion();
+    }
 
-        if (!client.doesBucketExist(BUCKET_NAME)) {
-            fail("Setup incomplete, tests will fail");
+    protected static void setSecureProperties(final TestRunner runner, final PropertyDescriptor serviceDescriptor) throws InitializationException {
+        if (runner.getProcessContext().getProperty(serviceDescriptor).isSet()) {
+            return;
         }
+
+        final AWSCredentialsProviderControllerService creds = new AWSCredentialsProviderControllerService();
+        runner.addControllerService("creds", creds);
+        runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, localstack.getAccessKey());
+        runner.setProperty(CredentialPropertyDescriptors.SECRET_KEY, localstack.getSecretKey());
+        runner.enableControllerService(creds);
+
+        runner.setProperty(serviceDescriptor, "creds");
+    }
+
+    @BeforeEach
+    public void clearKeys() {
+        addedKeys.clear();
     }
 
     @AfterAll
@@ -144,14 +166,13 @@ public abstract class AbstractS3IT {
             System.err.println("Unable to delete bucket " + BUCKET_NAME + e.toString());
         }
 
-        if (client.doesBucketExist(BUCKET_NAME)) {
+        if (client.doesBucketExistV2(BUCKET_NAME)) {
             fail("Incomplete teardown, subsequent tests might fail");
         }
     }
 
     protected void putTestFile(String key, File file) throws AmazonS3Exception {
         PutObjectRequest putRequest = new PutObjectRequest(BUCKET_NAME, key, file);
-
         client.putObject(putRequest);
     }
 
@@ -171,10 +192,27 @@ public abstract class AbstractS3IT {
         client.putObject(putRequest);
     }
 
+    protected void waitForFilesAvailable() {
+        for (final String key : addedKeys) {
+            final long maxWaitTimestamp = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10L);
+            while (System.currentTimeMillis() < maxWaitTimestamp) {
+                try {
+                    client.getObject(BUCKET_NAME, key);
+                } catch (final Exception e) {
+                    try {
+                        Thread.sleep(100L);
+                    } catch (final InterruptedException ie) {
+                        throw new AssertionError("Interrupted while waiting for files to become available", e);
+                    }
+                }
+            }
+        }
+    }
+
     protected void putFileWithObjectTag(String key, File file, List<Tag> objectTags) {
         PutObjectRequest putRequest = new PutObjectRequest(BUCKET_NAME, key, file);
         putRequest.setTagging(new ObjectTagging(objectTags));
-        PutObjectResult result = client.putObject(putRequest);
+        client.putObject(putRequest);
     }
 
     protected Path getResourcePath(String resourceName) {
@@ -210,8 +248,21 @@ public abstract class AbstractS3IT {
         return dekResult.getKeyId();
     }
 
-    protected static void deleteKMSKey(String keyId) {
-        ScheduleKeyDeletionRequest req = new ScheduleKeyDeletionRequest().withKeyId(keyId).withPendingWindowInDays(7);
-        kmsClient.scheduleKeyDeletion(req);
+
+    protected TestRunner initRunner(final Class<? extends AbstractS3Processor> processorClass) {
+        TestRunner runner = TestRunners.newTestRunner(processorClass);
+
+        try {
+            setSecureProperties(runner, AbstractS3Processor.AWS_CREDENTIALS_PROVIDER_SERVICE);
+        } catch (InitializationException e) {
+            Assertions.fail("Could not set security properties");
+        }
+
+        runner.setProperty(AbstractS3Processor.S3_REGION, getRegion());
+        runner.setProperty(AbstractS3Processor.ENDPOINT_OVERRIDE, getEndpointOverride());
+        runner.setProperty(AbstractS3Processor.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
+
+        return runner;
     }
+
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITDeleteS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITDeleteS3Object.java
index 1f4154703d..36beaf4185 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITDeleteS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITDeleteS3Object.java
@@ -16,13 +16,9 @@
  */
 package org.apache.nifi.processors.aws.s3;
 
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
-import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
 import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.Test;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -33,15 +29,11 @@ import java.util.Map;
 public class ITDeleteS3Object extends AbstractS3IT {
 
     @Test
-    public void testSimpleDelete() throws IOException {
+    public void testSimpleDelete() {
         // Prepares for this test
         putTestFile("delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
-
-        runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(DeleteS3Object.S3_REGION, REGION);
-        runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(DeleteS3Object.class);
 
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "delete-me");
@@ -53,16 +45,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testDeleteFolder() throws IOException {
+    public void testDeleteFolder() {
         // Prepares for this test
         putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
-
-        runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(DeleteS3Object.S3_REGION, REGION);
-        runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(DeleteS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "folder/delete-me");
         runner.enqueue(new byte[0], attrs);
@@ -73,24 +60,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testDeleteFolderUsingCredentialsProviderService() throws Throwable {
+    public void testDeleteFolderUsingCredentialsProviderService() {
         // Prepares for this test
         putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
-        final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
-
-        runner.addControllerService("awsCredentialsProvider", serviceImpl);
-
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, System.getProperty("user.home") + "/aws-credentials.properties");
-        runner.enableControllerService(serviceImpl);
-
-        runner.assertValid(serviceImpl);
-
-        runner.setProperty(DeleteS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
-        runner.setProperty(DeleteS3Object.S3_REGION, REGION);
-        runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(DeleteS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "folder/delete-me");
         runner.enqueue(new byte[0], attrs);
@@ -101,15 +75,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testDeleteFolderNoExpressionLanguage() throws IOException {
+    public void testDeleteFolderNoExpressionLanguage() {
         // Prepares for this test
         putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
-
-        runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(DeleteS3Object.S3_REGION, REGION);
-        runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(DeleteS3Object.class);
         runner.setProperty(DeleteS3Object.KEY, "folder/delete-me");
 
         final Map<String, String> attrs = new HashMap<>();
@@ -122,13 +92,8 @@ public class ITDeleteS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testTryToDeleteNotExistingFile() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
-
-        runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(DeleteS3Object.S3_REGION, REGION);
-        runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
-
+    public void testTryToDeleteNotExistingFile() {
+        final TestRunner runner = initRunner(DeleteS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "no-such-a-file");
         runner.enqueue(new byte[0], attrs);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITFetchS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITFetchS3Object.java
index 1b5d073d91..271cbc5fa7 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITFetchS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITFetchS3Object.java
@@ -17,8 +17,7 @@
 package org.apache.nifi.processors.aws.s3;
 
 import com.amazonaws.services.s3.model.ObjectMetadata;
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
-import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
+import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -36,14 +35,10 @@ import java.util.Map;
  */
 public class ITFetchS3Object extends AbstractS3IT {
     @Test
-    public void testSimpleGet() throws IOException {
+    public void testSimpleGet() throws IOException, InitializationException {
         putTestFile("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(FetchS3Object.class);
 
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "test-file");
@@ -59,15 +54,10 @@ public class ITFetchS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testSimpleGetEncrypted() throws IOException {
+    public void testSimpleGetEncrypted() throws IOException, InitializationException {
         putTestFileEncrypted("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(FetchS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "test-file");
         runner.enqueue(new byte[0], attrs);
@@ -85,20 +75,7 @@ public class ITFetchS3Object extends AbstractS3IT {
     public void testFetchS3ObjectUsingCredentialsProviderService() throws Throwable {
         putTestFile("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
-
-        runner.addControllerService("awsCredentialsProvider", serviceImpl);
-
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, System.getProperty("user.home") + "/aws-credentials.properties");
-        runner.enableControllerService(serviceImpl);
-        runner.assertValid(serviceImpl);
-
-        runner.setProperty(FetchS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(FetchS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "test-file");
         runner.enqueue(new byte[0], attrs);
@@ -110,12 +87,12 @@ public class ITFetchS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testTryToFetchNotExistingFile() {
+    public void testTryToFetchNotExistingFile() throws InitializationException {
         final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
 
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
+        setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
+        runner.setProperty(FetchS3Object.S3_REGION, getRegion());
+        runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
 
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "no-such-a-file");
@@ -127,16 +104,11 @@ public class ITFetchS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testContentsOfFileRetrieved() throws IOException {
+    public void testContentsOfFileRetrieved() throws IOException, InitializationException {
         String key = "folder/1.txt";
         putTestFile(key, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(FetchS3Object.class);
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", key);
         runner.enqueue(new byte[0], attrs);
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITListS3.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITListS3.java
index d89b3a8870..91399d098a 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITListS3.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITListS3.java
@@ -17,11 +17,9 @@
 package org.apache.nifi.processors.aws.s3;
 
 import com.amazonaws.services.s3.model.Tag;
-import org.apache.nifi.processors.aws.AbstractAWSProcessor;
-import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
+import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.Test;
 
 import java.io.FileNotFoundException;
@@ -37,18 +35,13 @@ import java.util.Map;
  */
 public class ITListS3 extends AbstractS3IT {
     @Test
-    public void testSimpleList() throws IOException {
+    public void testSimpleList() throws IOException, InitializationException {
         putTestFile("a", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         putTestFile("b/c", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(ListS3.class);
         runner.run();
 
         runner.assertAllFlowFilesTransferred(ListS3.REL_SUCCESS, 3);
@@ -65,20 +58,7 @@ public class ITListS3 extends AbstractS3IT {
         putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
-
-        runner.addControllerService("awsCredentialsProvider", serviceImpl);
-
-        runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.enableControllerService(serviceImpl);
-        runner.assertValid(serviceImpl);
-
-        runner.setProperty(ListS3.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initRunner(ListS3.class);
         runner.run();
 
         runner.assertAllFlowFilesTransferred(ListS3.REL_SUCCESS, 3);
@@ -95,11 +75,7 @@ public class ITListS3 extends AbstractS3IT {
         putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(ListS3.class);
         runner.setProperty(ListS3.DELIMITER, "/");
 
         runner.run();
@@ -116,11 +92,7 @@ public class ITListS3 extends AbstractS3IT {
         putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(ListS3.class);
         runner.setProperty(ListS3.PREFIX, "b/");
 
         runner.run();
@@ -137,11 +109,7 @@ public class ITListS3 extends AbstractS3IT {
         putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
+        final TestRunner runner = initRunner(ListS3.class);
         runner.setProperty(ListS3.PREFIX, "b/");
         runner.setProperty(ListS3.USE_VERSIONS, "true");
 
@@ -153,7 +121,7 @@ public class ITListS3 extends AbstractS3IT {
     }
 
     @Test
-    public void testObjectTagsWritten() {
+    public void testObjectTagsWritten() throws InitializationException {
         List<Tag> objectTags = new ArrayList<>();
         objectTags.add(new Tag("dummytag1", "dummyvalue1"));
         objectTags.add(new Tag("dummytag2", "dummyvalue2"));
@@ -161,12 +129,9 @@ public class ITListS3 extends AbstractS3IT {
         putFileWithObjectTag("t/fileWithTag", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), objectTags);
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
+        final TestRunner runner = initRunner(ListS3.class);
         runner.setProperty(ListS3.PREFIX, "t/");
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
+        runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(ListS3.WRITE_OBJECT_TAGS, "true");
 
         runner.run();
@@ -183,7 +148,7 @@ public class ITListS3 extends AbstractS3IT {
     }
 
     @Test
-    public void testUserMetadataWritten() throws FileNotFoundException {
+    public void testUserMetadataWritten() throws FileNotFoundException, InitializationException {
         Map<String, String> userMetadata = new HashMap<>();
         userMetadata.put("dummy.metadata.1", "dummyvalue1");
         userMetadata.put("dummy.metadata.2", "dummyvalue2");
@@ -191,12 +156,9 @@ public class ITListS3 extends AbstractS3IT {
         putFileWithUserMetadata("m/fileWithUserMetadata", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), userMetadata);
         waitForFilesAvailable();
 
-        final TestRunner runner = TestRunners.newTestRunner(new ListS3());
-
-        runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
+        final TestRunner runner = initRunner(ListS3.class);
         runner.setProperty(ListS3.PREFIX, "m/");
-        runner.setProperty(ListS3.REGION, REGION);
-        runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
+        runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(ListS3.WRITE_USER_METADATA, "true");
 
         runner.run();
@@ -212,12 +174,4 @@ public class ITListS3 extends AbstractS3IT {
         flowFiles.assertAttributeEquals("s3.user.metadata.dummy.metadata.2", "dummyvalue2");
     }
 
-    private void waitForFilesAvailable() {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
index 74f3b4febb..dab4f1fcde 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java
@@ -16,43 +16,29 @@
  */
 package org.apache.nifi.processors.aws.s3;
 
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.GetObjectTaggingRequest;
 import com.amazonaws.services.s3.model.GetObjectTaggingResult;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
 import com.amazonaws.services.s3.model.MultipartUpload;
 import com.amazonaws.services.s3.model.MultipartUploadListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PartETag;
-import com.amazonaws.services.s3.model.Region;
 import com.amazonaws.services.s3.model.StorageClass;
 import com.amazonaws.services.s3.model.Tag;
 import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
-import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
 import org.apache.nifi.processors.aws.s3.encryption.StandardS3EncryptionService;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.MockPropertyValue;
 import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
@@ -67,28 +53,25 @@ import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
 
-import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 /**
- * Provides integration level testing with actual AWS S3 resources for {@link PutS3Object} and requires additional configuration and resources to work.
+ * Makes use of Localstack TestContainer in order to test S3 integrations
  */
 public class ITPutS3Object extends AbstractS3IT {
-    private static final Logger logger = LoggerFactory.getLogger(ITPutS3Object.class);
 
-    final static String TEST_ENDPOINT = "https://endpoint.com";
-    //    final static String TEST_TRANSIT_URI = "https://" + BUCKET_NAME + ".endpoint.com";
     final static String TEST_PARTSIZE_STRING = "50 mb";
     final static Long   TEST_PARTSIZE_LONG = 50L * 1024L * 1024L;
 
     final static Long S3_MINIMUM_PART_SIZE = 50L * 1024L * 1024L;
     final static Long S3_MAXIMUM_OBJECT_SIZE = 5L * 1024L * 1024L * 1024L;
 
-    final static Pattern reS3ETag = Pattern.compile("[0-9a-fA-f]{32,32}(-[0-9]+)?");
+    final static Pattern reS3ETag = Pattern.compile("[0-9a-fA-f]{32}(-[0-9]+)?");
 
 
     private static String kmsKeyId = "";
@@ -105,12 +88,6 @@ public class ITPutS3Object extends AbstractS3IT {
         kmsKeyId = getKMSKey();
     }
 
-    @AfterAll
-    public static void teardownClass() {
-        if (StringUtils.isNotEmpty(kmsKeyId)) {
-            deleteKMSKey(kmsKeyId);
-        }
-    }
 
     @Test
     public void testSimplePut() throws IOException {
@@ -163,7 +140,7 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
     }
 
-    private void testPutThenFetch(String sseAlgorithm) throws IOException {
+    private void testPutThenFetch(String sseAlgorithm) throws IOException, InitializationException {
 
         // Put
         TestRunner runner = initTestRunner();
@@ -186,14 +163,8 @@ public class ITPutS3Object extends AbstractS3IT {
         }
 
         // Fetch
-        runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
-
+        runner = initFetchRunner();
         runner.enqueue(new byte[0], attrs);
-
         runner.run(1);
 
         runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
@@ -201,42 +172,27 @@ public class ITPutS3Object extends AbstractS3IT {
         MockFlowFile ff = ffs.get(0);
         ff.assertContentEquals(getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
 
-        if(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION.equals(sseAlgorithm)){
+        if (ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION.equals(sseAlgorithm)) {
             ff.assertAttributeEquals(PutS3Object.S3_SSE_ALGORITHM, ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION);
         } else {
             ff.assertAttributeNotExists(PutS3Object.S3_SSE_ALGORITHM);
         }
-
     }
 
     @Test
-    public void testPutThenFetchWithoutSSE() throws IOException {
+    public void testPutThenFetchWithoutSSE() throws IOException, InitializationException {
         testPutThenFetch(PutS3Object.NO_SERVER_SIDE_ENCRYPTION);
     }
 
     @Test
-    public void testPutThenFetchWithSSE() throws IOException {
+    public void testPutThenFetchWithSSE() throws IOException, InitializationException {
         testPutThenFetch(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION);
     }
 
 
     @Test
     public void testPutS3ObjectUsingCredentialsProviderService() throws Throwable {
-        final TestRunner runner = TestRunners.newTestRunner(new PutS3Object());
-
-        final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
-
-        runner.addControllerService("awsCredentialsProvider", serviceImpl);
-
-        runner.setProperty(serviceImpl, AbstractAWSCredentialsProviderProcessor.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.enableControllerService(serviceImpl);
-
-        runner.assertValid(serviceImpl);
-
-        runner.setProperty(PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
-
+        final TestRunner runner = initTestRunner();
         assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
 
         for (int i = 0; i < 3; i++) {
@@ -247,21 +203,15 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.run(3);
 
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 3);
-
     }
 
     @Test
-    public void testMetaData() throws IOException {
-        PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
-
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
-        PropertyDescriptor prop1 = processor.getSupportedDynamicPropertyDescriptor("TEST-PROP-1");
-        runner.setProperty(prop1, "TESTING-1-2-3");
-        PropertyDescriptor prop2 = processor.getSupportedDynamicPropertyDescriptor("TEST-PROP-2");
-        runner.setProperty(prop2, "TESTING-4-5-6");
+    public void testMetaData() throws IOException, InitializationException {
+        final TestRunner runner = initTestRunner();
+
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
+        runner.setProperty("TEST-PROP-1", "TESTING-1-2-3");
+        runner.setProperty("TEST-PROP-2", "TESTING-4-5-6");
 
         final Map<String, String> attrs = new HashMap<>();
         attrs.put("filename", "meta.txt");
@@ -308,7 +258,7 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testContentDispositionNull() throws IOException {
+    public void testContentDispositionNull() throws IOException, InitializationException {
         // Put
         TestRunner runner = initTestRunner();
 
@@ -320,14 +270,8 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
 
         // Fetch
-        runner = TestRunners.newTestRunner(new FetchS3Object());
-
-        runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(FetchS3Object.S3_REGION, REGION);
-        runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
-
+        runner = initFetchRunner();
         runner.enqueue(new byte[0], attrs);
-
         runner.run(1);
 
         runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
@@ -339,17 +283,14 @@ public class ITPutS3Object extends AbstractS3IT {
 
     @Test
     public void testContentDispositionAttachment() throws IOException {
-        TestRunner runner = initTestRunner();
-
+        final TestRunner runner = initTestRunner();
         runner.setProperty(PutS3Object.CONTENT_DISPOSITION, PutS3Object.CONTENT_DISPOSITION_ATTACHMENT);
-
         runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
-
         runner.run();
 
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
-        MockFlowFile ff1 = flowFiles.get(0);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
+        final MockFlowFile ff1 = flowFiles.get(0);
         ff1.assertAttributeEquals(PutS3Object.S3_CONTENT_DISPOSITION, "attachment; filename=\"hello.txt\"");
     }
 
@@ -384,64 +325,6 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
     }
 
-    @Test
-    public void testStorageClasses() throws IOException {
-        TestRunner runner = initTestRunner();
-
-        assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
-
-        for (StorageClass storageClass : StorageClass.values()) {
-            if (storageClass == StorageClass.Outposts) {
-                // Outposts storage class cannot be tested on AWS cloud infrastructure
-                continue;
-            }
-
-            runner.setProperty(PutS3Object.STORAGE_CLASS, storageClass.name());
-
-            final Map<String, String> attrs = new HashMap<>();
-            attrs.put("filename", "testStorageClasses/small_" + storageClass.name() + ".txt");
-            runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
-
-            runner.run();
-
-            runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
-            FlowFile file = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS).get(0);
-            assertEquals(storageClass.toString(), file.getAttribute(PutS3Object.S3_STORAGECLASS_ATTR_KEY));
-
-            runner.clearTransferState();
-        }
-    }
-
-    @Test
-    public void testStorageClassesMultipart() {
-        TestRunner runner = initTestRunner();
-
-        runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, "50 MB");
-        runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, "50 MB");
-
-        assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
-
-        for (StorageClass storageClass : StorageClass.values()) {
-            if (storageClass == StorageClass.Outposts) {
-                // Outposts storage class cannot be tested on AWS cloud infrastructure
-                continue;
-            }
-
-            runner.setProperty(PutS3Object.STORAGE_CLASS, storageClass.name());
-
-            final Map<String, String> attrs = new HashMap<>();
-            attrs.put("filename", "testStorageClasses/large_" + storageClass.name() + ".dat");
-            runner.enqueue(new byte[50 * 1024 * 1024 + 1], attrs);
-
-            runner.run();
-
-            runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
-            FlowFile file = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS).get(0);
-            assertEquals(storageClass.toString(), file.getAttribute(PutS3Object.S3_STORAGECLASS_ATTR_KEY));
-
-            runner.clearTransferState();
-        }
-    }
 
     @Test
     public void testPermissions() throws IOException {
@@ -459,19 +342,15 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testDynamicProperty() {
+    public void testDynamicProperty() throws InitializationException {
         final String DYNAMIC_ATTRIB_KEY = "fs.runTimestamp";
         final String DYNAMIC_ATTRIB_VALUE = "${now():toNumber()}";
 
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
 
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
-        PropertyDescriptor testAttrib = processor.getSupportedDynamicPropertyDescriptor(DYNAMIC_ATTRIB_KEY);
-        runner.setProperty(testAttrib, DYNAMIC_ATTRIB_VALUE);
+        runner.setProperty(DYNAMIC_ATTRIB_KEY, DYNAMIC_ATTRIB_VALUE);
 
         final String FILE1_NAME = "file1";
         Map<String, String> attribs = new HashMap<>();
@@ -479,7 +358,6 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.enqueue("123".getBytes(), attribs);
 
         runner.assertValid();
-        processor.getPropertyDescriptor(DYNAMIC_ATTRIB_KEY);
         runner.run();
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS);
         final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
@@ -498,15 +376,14 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testProvenance() {
+    public void testProvenance() throws InitializationException {
         final String PROV1_FILE = "provfile1";
 
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
 
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
+        setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
+        runner.setProperty(PutS3Object.S3_REGION, getRegion());
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(PutS3Object.KEY, "${filename}");
 
         Map<String, String> attribs = new HashMap<>();
@@ -523,9 +400,8 @@ public class ITPutS3Object extends AbstractS3IT {
         assertEquals(1, provenanceEvents.size());
         ProvenanceEventRecord provRec1 = provenanceEvents.get(0);
         assertEquals(ProvenanceEventType.SEND, provRec1.getEventType());
-        assertEquals(processor.getIdentifier(), provRec1.getComponentId());
-        client.setRegion(Region.fromValue(REGION).toAWSRegion());
-        String targetUri = client.getUrl(BUCKET_NAME, PROV1_FILE).toString();
+        assertEquals(runner.getProcessor().getIdentifier(), provRec1.getComponentId());
+        String targetUri = getClient().getUrl(BUCKET_NAME, PROV1_FILE).toString();
         assertEquals(targetUri, provRec1.getTransitUri());
         assertEquals(8, provRec1.getUpdatedAttributes().size());
         assertEquals(BUCKET_NAME, provRec1.getUpdatedAttributes().get(PutS3Object.S3_BUCKET_KEY));
@@ -559,42 +435,16 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testEndpointOverride() {
-        // remove leading "/" from filename to avoid duplicate separators
-        final String TESTKEY = AbstractS3IT.SAMPLE_FILE_RESOURCE_NAME.substring(1);
-
-        final PutS3Object processor = new TestablePutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+    public void testMultipartProperties() throws InitializationException {
+        final TestRunner runner = initTestRunner();
         final ProcessContext context = runner.getProcessContext();
 
-        runner.setProperty(PutS3Object.ENDPOINT_OVERRIDE, TEST_ENDPOINT);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
-        runner.setProperty(PutS3Object.KEY, TESTKEY);
-
-        runner.run();
-
-        assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET).toString());
-        assertEquals(TESTKEY, context.getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).toString());
-        assertEquals(TEST_ENDPOINT, context.getProperty(PutS3Object.ENDPOINT_OVERRIDE).toString());
-
-        String s3url = ((TestablePutS3Object)processor).testable_getClient(context).getResourceUrl(BUCKET_NAME, TESTKEY);
-        assertEquals(TEST_ENDPOINT + "/" + BUCKET_NAME + "/" + TESTKEY, s3url);
-    }
-
-    @Test
-    public void testMultipartProperties() {
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
-        final ProcessContext context = runner.getProcessContext();
-
-        runner.setProperty(PutS3Object.FULL_CONTROL_USER_LIST,
-                "28545acd76c35c7e91f8409b95fd1aa0c0914bfa1ac60975d9f48bc3c5e090b5");
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
+        runner.setProperty(PutS3Object.FULL_CONTROL_USER_LIST, "28545acd76c35c7e91f8409b95fd1aa0c0914bfa1ac60975d9f48bc3c5e090b5");
         runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(PutS3Object.KEY, AbstractS3IT.SAMPLE_FILE_RESOURCE_NAME);
 
-        assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET).toString());
+        assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).toString());
         assertEquals(SAMPLE_FILE_RESOURCE_NAME, context.getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).toString());
         assertEquals(TEST_PARTSIZE_LONG.longValue(),
                 context.getProperty(PutS3Object.MULTIPART_PART_SIZE).asDataSize(DataUnit.B).longValue());
@@ -602,10 +452,9 @@ public class ITPutS3Object extends AbstractS3IT {
 
     @Test
     public void testLocalStatePersistence() throws IOException {
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
 
-        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
+        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
         final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
         final String cacheKey1 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key;
         final String cacheKey2 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-v2";
@@ -615,6 +464,7 @@ public class ITPutS3Object extends AbstractS3IT {
          * store 3 versions of state
          */
         PutS3Object.MultipartState state1orig = new PutS3Object.MultipartState();
+        final PutS3Object processor = (PutS3Object) runner.getProcessor();
         processor.persistLocalState(cacheKey1, state1orig);
 
         PutS3Object.MultipartState state2orig = new PutS3Object.MultipartState();
@@ -642,8 +492,8 @@ public class ITPutS3Object extends AbstractS3IT {
         uploadList.add(upload3);
         final MultipartUploadListing uploadListing = new MultipartUploadListing();
         uploadListing.setMultipartUploads(uploadList);
-        final MockAmazonS3Client mockClient = new MockAmazonS3Client();
-        mockClient.setListing(uploadListing);
+        final AmazonS3 mockClient = mock(AmazonS3.class);
+        when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
 
         /*
          * reload and validate stored state
@@ -675,10 +525,10 @@ public class ITPutS3Object extends AbstractS3IT {
 
     @Test
     public void testStatePersistsETags() throws IOException {
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
+        final PutS3Object processor = (PutS3Object) runner.getProcessor();
 
-        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
+        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
         final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
         final String cacheKey1 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-bv1";
         final String cacheKey2 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-bv2";
@@ -732,8 +582,8 @@ public class ITPutS3Object extends AbstractS3IT {
         uploadList.add(upload2);
         final MultipartUploadListing uploadListing = new MultipartUploadListing();
         uploadListing.setMultipartUploads(uploadList);
-        final MockAmazonS3Client mockClient = new MockAmazonS3Client();
-        mockClient.setListing(uploadListing);
+        final AmazonS3 mockClient = mock(AmazonS3.class);
+        when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
 
         /*
          * load state and validate that
@@ -751,10 +601,10 @@ public class ITPutS3Object extends AbstractS3IT {
 
     @Test
     public void testStateRemove() throws IOException {
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
+        final PutS3Object processor = (PutS3Object) runner.getProcessor();
 
-        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
+        final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
         final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
         final String cacheKey = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-sr";
 
@@ -765,8 +615,8 @@ public class ITPutS3Object extends AbstractS3IT {
         uploadList.add(upload1);
         final MultipartUploadListing uploadListing = new MultipartUploadListing();
         uploadListing.setMultipartUploads(uploadList);
-        final MockAmazonS3Client mockClient = new MockAmazonS3Client();
-        mockClient.setListing(uploadListing);
+        final AmazonS3 mockClient = mock(AmazonS3.class);
+        when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
 
         /*
          * store state, retrieve and validate, remove and validate
@@ -786,7 +636,7 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testMultipartSmallerThanMinimum() throws IOException {
+    public void testMultipartSmallerThanMinimum() throws IOException, InitializationException {
         final String FILE1_NAME = "file1";
 
         final byte[] megabyte = new byte[1024 * 1024];
@@ -804,12 +654,9 @@ public class ITPutS3Object extends AbstractS3IT {
         assertTrue(megabyte.length < S3_MINIMUM_PART_SIZE);
         assertTrue(TEST_PARTSIZE_LONG >= S3_MINIMUM_PART_SIZE && TEST_PARTSIZE_LONG <= S3_MAXIMUM_OBJECT_SIZE);
 
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
 
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
 
         Map<String, String> attributes = new HashMap<>();
@@ -833,7 +680,7 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
-    public void testMultipartBetweenMinimumAndMaximum() throws IOException {
+    public void testMultipartBetweenMinimumAndMaximum() throws IOException, InitializationException {
         final String FILE1_NAME = "file1";
 
         final byte[] megabyte = new byte[1024 * 1024];
@@ -849,12 +696,11 @@ public class ITPutS3Object extends AbstractS3IT {
         assertTrue(tempByteCount > S3_MINIMUM_PART_SIZE && tempByteCount < S3_MAXIMUM_OBJECT_SIZE);
         assertTrue(tempByteCount > TEST_PARTSIZE_LONG);
 
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
+        final TestRunner runner = initTestRunner();
 
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
+        setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
+        runner.setProperty(PutS3Object.S3_REGION, getRegion());
+        runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
         runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, TEST_PARTSIZE_STRING);
         runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
 
@@ -878,93 +724,6 @@ public class ITPutS3Object extends AbstractS3IT {
         assertEquals(tempByteCount, ff1.getSize());
     }
 
-    @Test
-    public void testMultipartLargerThanObjectMaximum() throws IOException {
-        final String FILE1_NAME = "file1";
-
-        final byte[] megabyte = new byte[1024 * 1024];
-        final Path tempFile = Files.createTempFile("s3multipart", "tmp");
-        final FileOutputStream tempOut = new FileOutputStream(tempFile.toFile());
-        for (int i = 0; i < (S3_MAXIMUM_OBJECT_SIZE / 1024 / 1024 + 1); i++) {
-            tempOut.write(megabyte);
-        }
-        tempOut.close();
-
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
-
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
-        runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
-
-        Map<String, String> attributes = new HashMap<>();
-        attributes.put(CoreAttributes.FILENAME.key(), FILE1_NAME);
-        runner.enqueue(new FileInputStream(tempFile.toFile()), attributes);
-
-        runner.assertValid();
-        runner.run();
-        runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS);
-        final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
-        assertEquals(1, successFiles.size());
-        final List<MockFlowFile> failureFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_FAILURE);
-        assertEquals(0, failureFiles.size());
-        MockFlowFile ff1 = successFiles.get(0);
-        assertEquals(FILE1_NAME, ff1.getAttribute(CoreAttributes.FILENAME.key()));
-        assertEquals(BUCKET_NAME, ff1.getAttribute(PutS3Object.S3_BUCKET_KEY));
-        assertEquals(FILE1_NAME, ff1.getAttribute(PutS3Object.S3_OBJECT_KEY));
-        assertTrue(reS3ETag.matcher(ff1.getAttribute(PutS3Object.S3_ETAG_ATTR_KEY)).matches());
-        assertTrue(ff1.getSize() > S3_MAXIMUM_OBJECT_SIZE);
-    }
-
-    @Test
-    public void testS3MultipartAgeoff() throws InterruptedException, IOException {
-        final PutS3Object processor = new PutS3Object();
-        final TestRunner runner = TestRunners.newTestRunner(processor);
-        final ProcessContext context = runner.getProcessContext();
-
-        runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
-        runner.setProperty(PutS3Object.S3_REGION, REGION);
-        runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
-
-        // set check interval and age off to minimum values
-        runner.setProperty(PutS3Object.MULTIPART_S3_AGEOFF_INTERVAL, "1 milli");
-        runner.setProperty(PutS3Object.MULTIPART_S3_MAX_AGE, "1 milli");
-
-        // create some dummy uploads
-        for (int i = 0; i < 3; i++) {
-            final InitiateMultipartUploadRequest initiateRequest = new InitiateMultipartUploadRequest(
-                    BUCKET_NAME, "file" + i + ".txt");
-            assertDoesNotThrow(() -> client.initiateMultipartUpload(initiateRequest));
-        }
-
-        // Age off is time dependent, so test has some timing constraints.  This
-        // sleep() delays long enough to satisfy interval and age intervals.
-        Thread.sleep(2000L);
-
-        // System millis are used for timing, but it is incremented on each
-        // call to circumvent what appears to be caching in the AWS library.
-        // The increments are 1000 millis because AWS returns upload
-        // initiation times in whole seconds.
-        long now = System.currentTimeMillis();
-
-        MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now, BUCKET_NAME);
-        assertEquals(3, uploadList.getMultipartUploads().size());
-
-        MultipartUpload upload0 = uploadList.getMultipartUploads().get(0);
-        processor.abortS3MultipartUpload(client, BUCKET_NAME, upload0);
-
-        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000, BUCKET_NAME);
-        assertEquals(2, uploadList.getMultipartUploads().size());
-
-        final Map<String, String> attrs = new HashMap<>();
-        attrs.put("filename", "test-upload.txt");
-        runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
-        runner.run();
-
-        uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000, BUCKET_NAME);
-        assertEquals(0, uploadList.getMultipartUploads().size());
-    }
 
     @Test
     public void testObjectTags() throws IOException {
@@ -981,7 +740,7 @@ public class ITPutS3Object extends AbstractS3IT {
         runner.run();
         runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
 
-        GetObjectTaggingResult result = client.getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, "tag-test.txt"));
+        GetObjectTaggingResult result = getClient().getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, "tag-test.txt"));
         List<Tag> objectTags = result.getTagSet();
 
         for (Tag tag : objectTags) {
@@ -1028,12 +787,14 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
+    @Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
     public void testEncryptionServiceWithServerSideKMSEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
         byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
         testEncryptionServiceWithServerSideKMSEncryptionStrategy(smallData);
     }
 
     @Test
+    @Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
     public void testEncryptionServiceWithServerSideKMSEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
         byte[] largeData = new byte[51 * 1024 * 1024];
         testEncryptionServiceWithServerSideKMSEncryptionStrategy(largeData);
@@ -1062,12 +823,14 @@ public class ITPutS3Object extends AbstractS3IT {
     }
 
     @Test
+    @Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
     public void testEncryptionServiceWithServerSideCEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
         byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
         testEncryptionServiceWithServerSideCEncryptionStrategy(smallData);
     }
 
     @Test
+    @Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
     public void testEncryptionServiceWithServerSideCEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
         byte[] largeData = new byte[51 * 1024 * 1024];
         testEncryptionServiceWithServerSideCEncryptionStrategy(largeData);
@@ -1097,17 +860,6 @@ public class ITPutS3Object extends AbstractS3IT {
         flowFile.assertAttributeEquals(PutS3Object.S3_ENCRYPTION_STRATEGY, AmazonS3EncryptionService.STRATEGY_NAME_SSE_C);
     }
 
-    @Test
-    public void testEncryptionServiceWithClientSideKMSEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
-        byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
-        testEncryptionServiceWithClientSideKMSEncryptionStrategy(smallData);
-    }
-
-    @Test
-    public void testEncryptionServiceWithClientSideKMSEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
-        byte[] largeData = new byte[51 * 1024 * 1024];
-        testEncryptionServiceWithClientSideKMSEncryptionStrategy(largeData);
-    }
 
     private void testEncryptionServiceWithClientSideKMSEncryptionStrategy(byte[] data) throws InitializationException, IOException {
         TestRunner runner = createPutEncryptionTestRunner(AmazonS3EncryptionService.STRATEGY_NAME_CSE_KMS, kmsKeyId);
@@ -1162,16 +914,10 @@ public class ITPutS3Object extends AbstractS3IT {
         MockFlowFile flowFile = fetchEncryptedFlowFile(attrs, AmazonS3EncryptionService.STRATEGY_NAME_CSE_C, randomKeyMaterial);
         flowFile.assertAttributeEquals(PutS3Object.S3_ENCRYPTION_STRATEGY, AmazonS3EncryptionService.STRATEGY_NAME_CSE_C);
         flowFile.assertContentEquals(data);
-
-        flowFile.assertAttributeExists("x-amz-key");
-        flowFile.assertAttributeNotEquals("x-amz-key", "");
-
-        flowFile.assertAttributeExists("x-amz-iv");
-        flowFile.assertAttributeNotEquals("x-amz-iv", "");
     }
 
-    private static TestRunner createPutEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
-        TestRunner runner = createEncryptionTestRunner(new PutS3Object(), strategyName, keyIdOrMaterial);
+    private TestRunner createPutEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
+        TestRunner runner = createEncryptionTestRunner(PutS3Object.class, strategyName, keyIdOrMaterial);
 
         runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, "50 MB");
         runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, "50 MB");
@@ -1179,15 +925,15 @@ public class ITPutS3Object extends AbstractS3IT {
         return runner;
     }
 
-    private static TestRunner createFetchEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
+    private TestRunner createFetchEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
         if (strategyName.equals(AmazonS3EncryptionService.STRATEGY_NAME_SSE_S3) || strategyName.equals(AmazonS3EncryptionService.STRATEGY_NAME_SSE_KMS)) {
             strategyName = null;
         }
 
-        return createEncryptionTestRunner(new FetchS3Object(), strategyName, keyIdOrMaterial);
+        return createEncryptionTestRunner(FetchS3Object.class, strategyName, keyIdOrMaterial);
     }
 
-    private static MockFlowFile fetchEncryptedFlowFile(Map<String, String> attributes, String strategyName, String keyIdOrMaterial) throws InitializationException {
+    private MockFlowFile fetchEncryptedFlowFile(Map<String, String> attributes, String strategyName, String keyIdOrMaterial) throws InitializationException {
         final TestRunner runner = createFetchEncryptionTestRunner(strategyName, keyIdOrMaterial);
         runner.enqueue(new byte[0], attributes);
         runner.run(1);
@@ -1196,13 +942,8 @@ public class ITPutS3Object extends AbstractS3IT {
         return flowFiles.get(0);
     }
 
-    private static TestRunner createEncryptionTestRunner(Processor processor, String strategyName, String keyIdOrMaterial) throws InitializationException {
... 1402 lines suppressed ...