attributeDefinitions = new ArrayList<>();
- attributeDefinitions.add(new AttributeDefinition().withAttributeName(DYNAMODB_ATTRIBUTE_NAME).withAttributeType(ScalarAttributeType.S));
-
- CreateTableRequest createTableRequest = new CreateTableRequest(tableName, keySchema);
- createTableRequest.setAttributeDefinitions(attributeDefinitions);
- createTableRequest.setBillingMode(billingMode);
+ attributeDefinitions.add(AttributeDefinition.builder().attributeName(DYNAMODB_ATTRIBUTE_NAME).attributeType(ScalarAttributeType.S).build());
+ CreateTableRequest.Builder createTableRequestBuilder = CreateTableRequest.builder();
if (billingMode.equals(BillingMode.PROVISIONED.name())) {
- createTableRequest.setProvisionedThroughput(new ProvisionedThroughput()
- .withReadCapacityUnits(dynamoDBLockConfiguration.getLong(DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY))
- .withWriteCapacityUnits(dynamoDBLockConfiguration.getLong(DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY)));
+ createTableRequestBuilder.provisionedThroughput(ProvisionedThroughput.builder()
+ .readCapacityUnits(dynamoDBLockConfiguration.getLong(DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY))
+ .writeCapacityUnits(dynamoDBLockConfiguration.getLong(DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY))
+ .build());
}
- dynamoDB.createTable(createTableRequest);
+ createTableRequestBuilder.tableName(tableName)
+ .keySchema(keySchema)
+ .attributeDefinitions(attributeDefinitions)
+ .billingMode(billingMode);
+ dynamoDB.createTable(createTableRequestBuilder.build());
LOG.info("Creating dynamoDB table " + tableName + ", waiting for table to be active");
try {
- TableUtils.waitUntilActive(dynamoDB, tableName, dynamoDBLockConfiguration.getInt(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT), 20 * 1000);
- } catch (TableUtils.TableNeverTransitionedToStateException e) {
+
+ DynamoTableUtils.waitUntilActive(dynamoDB, tableName, dynamoDBLockConfiguration.getInt(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT), 20 * 1000);
+ } catch (DynamoTableUtils.TableNeverTransitionedToStateException e) {
throw new HoodieLockException("Created dynamoDB table never transits to active", e);
} catch (InterruptedException e) {
throw new HoodieLockException("Thread interrupted while waiting for dynamoDB table to turn active", e);
diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/utils/DynamoTableUtils.java b/hudi-aws/src/main/java/org/apache/hudi/aws/utils/DynamoTableUtils.java
new file mode 100644
index 0000000000000..b507ca2bf478c
--- /dev/null
+++ b/hudi-aws/src/main/java/org/apache/hudi/aws/utils/DynamoTableUtils.java
@@ -0,0 +1,268 @@
+/*
+ * 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.hudi.aws.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.ResourceInUseException;
+import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.dynamodb.model.TableDescription;
+import software.amazon.awssdk.services.dynamodb.model.TableStatus;
+
+/**
+ * Reused code from https://github.com/aws/aws-sdk-java-v2/blob/master/services/dynamodb/src/test/java/utils/test/util/TableUtils.java
+ *
+ * Utility methods for working with DynamoDB tables.
+ *
+ *
+ * // ... create DynamoDB table ...
+ * try {
+ * waitUntilActive(dynamoDB, myTableName());
+ * } catch (SdkClientException e) {
+ * // table didn't become active
+ * }
+ * // ... start making calls to table ...
+ *
+ */
+public class DynamoTableUtils {
+
+ private static final int DEFAULT_WAIT_TIMEOUT = 20 * 60 * 1000;
+ private static final int DEFAULT_WAIT_INTERVAL = 10 * 1000;
+ /**
+ * The logging utility.
+ */
+ private static final Logger LOGGER = LoggerFactory.getLogger(DynamoTableUtils.class);
+
+ /**
+ * Waits up to 10 minutes for a specified DynamoDB table to resolve,
+ * indicating that it exists. If the table doesn't return a result after
+ * this time, a SdkClientException is thrown.
+ *
+ * @param dynamo
+ * The DynamoDB client to use to make requests.
+ * @param tableName
+ * The name of the table being resolved.
+ *
+ * @throws SdkClientException
+ * If the specified table does not resolve before this method
+ * times out and stops polling.
+ * @throws InterruptedException
+ * If the thread is interrupted while waiting for the table to
+ * resolve.
+ */
+ public static void waitUntilExists(final DynamoDbClient dynamo, final String tableName)
+ throws InterruptedException {
+ waitUntilExists(dynamo, tableName, DEFAULT_WAIT_TIMEOUT, DEFAULT_WAIT_INTERVAL);
+ }
+
+ /**
+ * Waits up to a specified amount of time for a specified DynamoDB table to
+ * resolve, indicating that it exists. If the table doesn't return a result
+ * after this time, a SdkClientException is thrown.
+ *
+ * @param dynamo
+ * The DynamoDB client to use to make requests.
+ * @param tableName
+ * The name of the table being resolved.
+ * @param timeout
+ * The maximum number of milliseconds to wait.
+ * @param interval
+ * The poll interval in milliseconds.
+ *
+ * @throws SdkClientException
+ * If the specified table does not resolve before this method
+ * times out and stops polling.
+ * @throws InterruptedException
+ * If the thread is interrupted while waiting for the table to
+ * resolve.
+ */
+ public static void waitUntilExists(final DynamoDbClient dynamo, final String tableName, final int timeout,
+ final int interval) throws InterruptedException {
+ TableDescription table = waitForTableDescription(dynamo, tableName, null, timeout, interval);
+
+ if (table == null) {
+ throw SdkClientException.builder().message("Table " + tableName + " never returned a result").build();
+ }
+ }
+
+ /**
+ * Waits up to 10 minutes for a specified DynamoDB table to move into the
+ * ACTIVE state. If the table does not exist or does not
+ * transition to the ACTIVE state after this time, then
+ * SdkClientException is thrown.
+ *
+ * @param dynamo
+ * The DynamoDB client to use to make requests.
+ * @param tableName
+ * The name of the table whose status is being checked.
+ *
+ * @throws TableNeverTransitionedToStateException
+ * If the specified table does not exist or does not transition
+ * into the ACTIVE state before this method times
+ * out and stops polling.
+ * @throws InterruptedException
+ * If the thread is interrupted while waiting for the table to
+ * transition into the ACTIVE state.
+ */
+ public static void waitUntilActive(final DynamoDbClient dynamo, final String tableName)
+ throws InterruptedException, TableNeverTransitionedToStateException {
+ waitUntilActive(dynamo, tableName, DEFAULT_WAIT_TIMEOUT, DEFAULT_WAIT_INTERVAL);
+ }
+
+ /**
+ * Waits up to a specified amount of time for a specified DynamoDB table to
+ * move into the ACTIVE state. If the table does not exist or
+ * does not transition to the ACTIVE state after this time,
+ * then a SdkClientException is thrown.
+ *
+ * @param dynamo
+ * The DynamoDB client to use to make requests.
+ * @param tableName
+ * The name of the table whose status is being checked.
+ * @param timeout
+ * The maximum number of milliseconds to wait.
+ * @param interval
+ * The poll interval in milliseconds.
+ *
+ * @throws TableNeverTransitionedToStateException
+ * If the specified table does not exist or does not transition
+ * into the ACTIVE state before this method times
+ * out and stops polling.
+ * @throws InterruptedException
+ * If the thread is interrupted while waiting for the table to
+ * transition into the ACTIVE state.
+ */
+ public static void waitUntilActive(final DynamoDbClient dynamo, final String tableName, final int timeout,
+ final int interval) throws InterruptedException, TableNeverTransitionedToStateException {
+ TableDescription table = waitForTableDescription(dynamo, tableName, TableStatus.ACTIVE, timeout, interval);
+
+ if (table == null || !table.tableStatus().equals(TableStatus.ACTIVE)) {
+ throw new TableNeverTransitionedToStateException(tableName, TableStatus.ACTIVE);
+ }
+ }
+
+ /**
+ * Wait for the table to reach the desired status and returns the table
+ * description
+ *
+ * @param dynamo
+ * Dynamo client to use
+ * @param tableName
+ * Table name to poll status of
+ * @param desiredStatus
+ * Desired {@link TableStatus} to wait for. If null this method
+ * simply waits until DescribeTable returns something non-null
+ * (i.e. any status)
+ * @param timeout
+ * Timeout in milliseconds to continue to poll for desired status
+ * @param interval
+ * Time to wait in milliseconds between poll attempts
+ * @return Null if DescribeTables never returns a result, otherwise the
+ * result of the last poll attempt (which may or may not have the
+ * desired state)
+ * @throws {@link
+ * IllegalArgumentException} If timeout or interval is invalid
+ */
+ private static TableDescription waitForTableDescription(final DynamoDbClient dynamo, final String tableName,
+ TableStatus desiredStatus, final int timeout, final int interval)
+ throws InterruptedException, IllegalArgumentException {
+ if (timeout < 0) {
+ throw new IllegalArgumentException("Timeout must be >= 0");
+ }
+ if (interval <= 0 || interval >= timeout) {
+ throw new IllegalArgumentException("Interval must be > 0 and < timeout");
+ }
+ long startTime = System.currentTimeMillis();
+ long endTime = startTime + timeout;
+
+ TableDescription table = null;
+ while (System.currentTimeMillis() < endTime) {
+ try {
+ table = dynamo.describeTable(DescribeTableRequest.builder().tableName(tableName).build()).table();
+ if (desiredStatus == null || table.tableStatus().equals(desiredStatus)) {
+ return table;
+
+ }
+ } catch (ResourceNotFoundException rnfe) {
+ // ResourceNotFound means the table doesn't exist yet,
+ // so ignore this error and just keep polling.
+ }
+
+ Thread.sleep(interval);
+ }
+ return table;
+ }
+
+ /**
+ * Creates the table and ignores any errors if it already exists.
+ * @param dynamo The Dynamo client to use.
+ * @param createTableRequest The create table request.
+ * @return True if created, false otherwise.
+ */
+ public static boolean createTableIfNotExists(final DynamoDbClient dynamo, final CreateTableRequest createTableRequest) {
+ try {
+ dynamo.createTable(createTableRequest);
+ return true;
+ } catch (final ResourceInUseException e) {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Table " + createTableRequest.tableName() + " already exists", e);
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Deletes the table and ignores any errors if it doesn't exist.
+ * @param dynamo The Dynamo client to use.
+ * @param deleteTableRequest The delete table request.
+ * @return True if deleted, false otherwise.
+ */
+ public static boolean deleteTableIfExists(final DynamoDbClient dynamo, final DeleteTableRequest deleteTableRequest) {
+ try {
+ dynamo.deleteTable(deleteTableRequest);
+ return true;
+ } catch (final ResourceNotFoundException e) {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Table " + deleteTableRequest.tableName() + " does not exist", e);
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Thrown by {@link DynamoTableUtils} when a table never reaches a desired state
+ */
+ public static class TableNeverTransitionedToStateException extends SdkClientException {
+
+ private static final long serialVersionUID = 8920567021104846647L;
+
+ public TableNeverTransitionedToStateException(String tableName, TableStatus desiredStatus) {
+ super(SdkClientException.builder()
+ .message("Table " + tableName + " never transitioned to desired state of "
+ + desiredStatus.toString()));
+ }
+
+ }
+
+}
diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java
index ebe84a4225348..5639db0258254 100644
--- a/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java
@@ -28,8 +28,9 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
-import com.amazonaws.regions.RegionUtils;
-import com.amazonaws.services.dynamodbv2.model.BillingMode;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.regions.RegionMetadata;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
/**
* Hoodie Configs for Locks.
@@ -71,20 +72,22 @@ public static DynamoDbBasedLockConfig.Builder newBuilder() {
+ "Each Hudi dataset should has it's unique key so concurrent writers could refer to the same partition key."
+ " By default we use the Hudi table name specified to be the partition key");
- public static final ConfigProperty DYNAMODB_LOCK_REGION = ConfigProperty
- .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "region")
- .defaultValue("us-east-1")
- .markAdvanced()
- .sinceVersion("0.10.0")
- .withInferFunction(cfg -> {
- String regionFromEnv = System.getenv("AWS_REGION");
- if (regionFromEnv != null) {
- return Option.of(RegionUtils.getRegion(regionFromEnv).getName());
- }
- return Option.empty();
- })
- .withDocumentation("For DynamoDB based lock provider, the region used in endpoint for Amazon DynamoDB service."
- + " Would try to first get it from AWS_REGION environment variable. If not find, by default use us-east-1");
+ public static final ConfigProperty DYNAMODB_LOCK_REGION =
+ ConfigProperty.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "region")
+ .defaultValue("us-east-1")
+ .markAdvanced()
+ .sinceVersion("0.10.0")
+ .withInferFunction(
+ cfg -> {
+ String regionFromEnv = System.getenv("AWS_REGION");
+ if (regionFromEnv != null) {
+ return Option.of(RegionMetadata.of(Region.of(regionFromEnv)).id());
+ }
+ return Option.empty();
+ })
+ .withDocumentation(
+ "For DynamoDB based lock provider, the region used in endpoint for Amazon DynamoDB service."
+ + " Would try to first get it from AWS_REGION environment variable. If not find, by default use us-east-1");
public static final ConfigProperty DYNAMODB_LOCK_BILLING_MODE = ConfigProperty
.key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "billing_mode")
diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java
index 051fe81e8b0ff..7a5e776db8d53 100644
--- a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java
+++ b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java
@@ -18,11 +18,11 @@
package org.apache.hudi.aws;
-import com.amazonaws.auth.BasicSessionCredentials;
import org.apache.hudi.config.HoodieAWSConfig;
import org.apache.hudi.common.config.HoodieConfig;
import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -34,9 +34,9 @@ public void testGetAWSCredentials() {
cfg.setValue(HoodieAWSConfig.AWS_ACCESS_KEY, "random-access-key");
cfg.setValue(HoodieAWSConfig.AWS_SECRET_KEY, "random-secret-key");
cfg.setValue(HoodieAWSConfig.AWS_SESSION_TOKEN, "random-session-token");
- BasicSessionCredentials credentials = (BasicSessionCredentials) org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(cfg.getProps()).getCredentials();
- assertEquals("random-access-key", credentials.getAWSAccessKeyId());
- assertEquals("random-secret-key", credentials.getAWSSecretKey());
- assertEquals("random-session-token", credentials.getSessionToken());
+ AwsSessionCredentials credentials = (AwsSessionCredentials) org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(cfg.getProps()).resolveCredentials();
+ assertEquals("random-access-key", credentials.accessKeyId());
+ assertEquals("random-secret-key", credentials.secretAccessKey());
+ assertEquals("random-session-token", credentials.sessionToken());
}
}
diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java b/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java
index 85f551e6fda82..600dd57869599 100644
--- a/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java
+++ b/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java
@@ -18,11 +18,11 @@
package org.apache.hudi.aws.cloudwatch;
-import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsync;
-import com.amazonaws.services.cloudwatch.model.Dimension;
-import com.amazonaws.services.cloudwatch.model.MetricDatum;
-import com.amazonaws.services.cloudwatch.model.PutMetricDataRequest;
-import com.amazonaws.services.cloudwatch.model.PutMetricDataResult;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.cloudwatch.model.Dimension;
+import software.amazon.awssdk.services.cloudwatch.model.MetricDatum;
+import software.amazon.awssdk.services.cloudwatch.model.PutMetricDataRequest;
+import software.amazon.awssdk.services.cloudwatch.model.PutMetricDataResponse;
import com.codahale.metrics.Clock;
import com.codahale.metrics.Counter;
import com.codahale.metrics.ExponentiallyDecayingReservoir;
@@ -67,10 +67,10 @@ public class TestCloudWatchReporter {
MetricRegistry metricRegistry;
@Mock
- AmazonCloudWatchAsync cloudWatchAsync;
+ CloudWatchAsyncClient cloudWatchAsync;
@Mock
- CompletableFuture cloudWatchFuture;
+ CompletableFuture cloudWatchFuture;
@Captor
ArgumentCaptor putMetricDataRequestCaptor;
@@ -89,7 +89,7 @@ public void setup() {
.convertDurationsTo(TimeUnit.MILLISECONDS)
.build(cloudWatchAsync);
- Mockito.when(cloudWatchAsync.putMetricDataAsync(ArgumentMatchers.any())).thenReturn(cloudWatchFuture);
+ Mockito.when(cloudWatchAsync.putMetricData((PutMetricDataRequest) ArgumentMatchers.any())).thenReturn(cloudWatchFuture);
}
@Test
@@ -130,52 +130,54 @@ public void testReporter() {
// Since there are 6 metrics in total, and max datums per request is 2 we would expect 3 calls to CloudWatch
// with 2 datums in each
- Mockito.verify(cloudWatchAsync, Mockito.times(3)).putMetricDataAsync(putMetricDataRequestCaptor.capture());
- Assertions.assertEquals(NAMESPACE, putMetricDataRequestCaptor.getValue().getNamespace());
+ Mockito.verify(cloudWatchAsync, Mockito.times(3)).putMetricData(putMetricDataRequestCaptor.capture());
+ Assertions.assertEquals(NAMESPACE, putMetricDataRequestCaptor.getValue().namespace());
List putMetricDataRequests = putMetricDataRequestCaptor.getAllValues();
- putMetricDataRequests.forEach(request -> assertEquals(2, request.getMetricData().size()));
+ putMetricDataRequests.forEach(request -> assertEquals(2, request.metricData().size()));
- List metricDataBatch1 = putMetricDataRequests.get(0).getMetricData();
- assertEquals(PREFIX + ".gauge1", metricDataBatch1.get(0).getMetricName());
- assertEquals(Double.valueOf(gauge1.getValue()), metricDataBatch1.get(0).getValue());
- assertDimensions(metricDataBatch1.get(0).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE);
+ List metricDataBatch1 = putMetricDataRequests.get(0).metricData();
+ assertEquals(PREFIX + ".gauge1", metricDataBatch1.get(0).metricName());
+ assertEquals(Double.valueOf(gauge1.getValue()), metricDataBatch1.get(0).value());
+ assertDimensions(metricDataBatch1.get(0).dimensions(), DIMENSION_GAUGE_TYPE_VALUE);
- assertEquals(PREFIX + ".gauge2", metricDataBatch1.get(1).getMetricName());
- assertEquals(gauge2.getValue(), metricDataBatch1.get(1).getValue());
- assertDimensions(metricDataBatch1.get(1).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE);
+ assertEquals(PREFIX + ".gauge2", metricDataBatch1.get(1).metricName());
+ assertEquals(gauge2.getValue(), metricDataBatch1.get(1).value());
+ assertDimensions(metricDataBatch1.get(1).dimensions(), DIMENSION_GAUGE_TYPE_VALUE);
- List metricDataBatch2 = putMetricDataRequests.get(1).getMetricData();
- assertEquals(PREFIX + ".counter1", metricDataBatch2.get(0).getMetricName());
- assertEquals(counter1.getCount(), metricDataBatch2.get(0).getValue().longValue());
- assertDimensions(metricDataBatch2.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
+ List metricDataBatch2 = putMetricDataRequests.get(1).metricData();
+ assertEquals(PREFIX + ".counter1", metricDataBatch2.get(0).metricName());
+ assertEquals(counter1.getCount(), metricDataBatch2.get(0).value().longValue());
+ assertDimensions(metricDataBatch2.get(0).dimensions(), DIMENSION_COUNT_TYPE_VALUE);
- assertEquals(PREFIX + ".histogram1", metricDataBatch2.get(1).getMetricName());
- assertEquals(histogram1.getCount(), metricDataBatch2.get(1).getValue().longValue());
- assertDimensions(metricDataBatch2.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
+ assertEquals(PREFIX + ".histogram1", metricDataBatch2.get(1).metricName());
+ assertEquals(histogram1.getCount(), metricDataBatch2.get(1).value().longValue());
+ assertDimensions(metricDataBatch2.get(1).dimensions(), DIMENSION_COUNT_TYPE_VALUE);
- List metricDataBatch3 = putMetricDataRequests.get(2).getMetricData();
- assertEquals(PREFIX + ".meter1", metricDataBatch3.get(0).getMetricName());
- assertEquals(meter1.getCount(), metricDataBatch3.get(0).getValue().longValue());
- assertDimensions(metricDataBatch3.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
+ List metricDataBatch3 = putMetricDataRequests.get(2).metricData();
+ assertEquals(PREFIX + ".meter1", metricDataBatch3.get(0).metricName());
+ assertEquals(meter1.getCount(), metricDataBatch3.get(0).value().longValue());
+ assertDimensions(metricDataBatch3.get(0).dimensions(), DIMENSION_COUNT_TYPE_VALUE);
- assertEquals(PREFIX + ".timer1", metricDataBatch3.get(1).getMetricName());
- assertEquals(timer1.getCount(), metricDataBatch3.get(1).getValue().longValue());
- assertDimensions(metricDataBatch3.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE);
+ assertEquals(PREFIX + ".timer1", metricDataBatch3.get(1).metricName());
+ assertEquals(timer1.getCount(), metricDataBatch3.get(1).value().longValue());
+ assertDimensions(metricDataBatch3.get(1).dimensions(), DIMENSION_COUNT_TYPE_VALUE);
reporter.stop();
- Mockito.verify(cloudWatchAsync).shutdown();
+ Mockito.verify(cloudWatchAsync).close();
}
private void assertDimensions(List actualDimensions, String metricTypeDimensionVal) {
assertEquals(2, actualDimensions.size());
- Dimension expectedTableNameDimension = new Dimension()
- .withName(DIMENSION_TABLE_NAME_KEY)
- .withValue(TABLE_NAME);
- Dimension expectedMetricTypeDimension = new Dimension()
- .withName(DIMENSION_METRIC_TYPE_KEY)
- .withValue(metricTypeDimensionVal);
+ Dimension expectedTableNameDimension = Dimension.builder()
+ .name(DIMENSION_TABLE_NAME_KEY)
+ .value(TABLE_NAME)
+ .build();
+ Dimension expectedMetricTypeDimension = Dimension.builder()
+ .name(DIMENSION_METRIC_TYPE_KEY)
+ .value(metricTypeDimensionVal)
+ .build();
assertEquals(expectedTableNameDimension, actualDimensions.get(0));
assertEquals(expectedMetricTypeDimension, actualDimensions.get(1));
diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java
index d2ab0375e050c..4738617125957 100644
--- a/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java
+++ b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java
@@ -18,13 +18,13 @@
package org.apache.hudi.aws.transaction.integ;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSStaticCredentialsProvider;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.client.builder.AwsClientBuilder;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
-import com.amazonaws.services.dynamodbv2.model.BillingMode;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+
import org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider;
import org.apache.hudi.common.config.LockConfiguration;
import org.apache.hudi.config.DynamoDbBasedLockConfig;
@@ -33,6 +33,7 @@
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import java.net.URI;
import java.util.UUID;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
@@ -46,7 +47,7 @@
public class ITTestDynamoDBBasedLockProvider {
private static LockConfiguration lockConfiguration;
- private static AmazonDynamoDB dynamoDb;
+ private static DynamoDbClient dynamoDb;
private static final String TABLE_NAME_PREFIX = "testDDBTable-";
private static final String REGION = "us-east-2";
@@ -103,18 +104,19 @@ public void testUnlockWithoutLock() {
dynamoDbBasedLockProvider.unlock();
}
- private static AmazonDynamoDB getDynamoClientWithLocalEndpoint() {
+ private static DynamoDbClient getDynamoClientWithLocalEndpoint() {
String endpoint = System.getProperty("dynamodb-local.endpoint");
if (endpoint == null || endpoint.isEmpty()) {
throw new IllegalStateException("dynamodb-local.endpoint system property not set");
}
- return AmazonDynamoDBClientBuilder.standard()
- .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(endpoint, REGION))
- .withCredentials(getCredentials())
+ return DynamoDbClient.builder()
+ .region(Region.of(REGION))
+ .endpointOverride(URI.create(endpoint))
+ .credentialsProvider(getCredentials())
.build();
}
- private static AWSCredentialsProvider getCredentials() {
- return new AWSStaticCredentialsProvider(new BasicAWSCredentials("random-access-key", "random-secret-key"));
+ private static AwsCredentialsProvider getCredentials() {
+ return StaticCredentialsProvider.create(AwsBasicCredentials.create("random-access-key", "random-secret-key"));
}
}
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index 8bdb743584ffd..ccda05eeaeca7 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -136,6 +136,18 @@
hudi-aws
${project.version}