diff --git a/emr-dynamodb-hadoop/pom.xml b/emr-dynamodb-hadoop/pom.xml index 12e84758..313cc384 100644 --- a/emr-dynamodb-hadoop/pom.xml +++ b/emr-dynamodb-hadoop/pom.xml @@ -54,8 +54,13 @@ - com.amazonaws - aws-java-sdk-dynamodb + software.amazon.awssdk + dynamodb + + + + software.amazon.awssdk + apache-client @@ -89,6 +94,13 @@ 0 + + org.apache.maven.plugins + maven-assembly-plugin + + false + + diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueDeserializer.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueDeserializer.java new file mode 100644 index 00000000..307fbe3b --- /dev/null +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueDeserializer.java @@ -0,0 +1,120 @@ +/** + * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). You may not use this file + * except in compliance with the License. A copy of the License is located at + * + *     http://aws.amazon.com/apache2.0/ + * + * or in the "LICENSE.TXT" file accompanying this file. This file 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.hadoop.dynamodb; + +import com.google.gson.JsonArray; +import com.google.gson.JsonDeserializationContext; +import com.google.gson.JsonDeserializer; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParseException; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.dynamodb.type.DynamoDBTypeConstants; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; + +public class AttributeValueDeserializer implements JsonDeserializer { + + @Override + public AttributeValue deserialize(JsonElement jsonElement, Type type, + JsonDeserializationContext context) throws JsonParseException { + if (jsonElement.isJsonNull()) { + return null; + } + + if (jsonElement.isJsonObject()) { + JsonObject jsonObject = jsonElement.getAsJsonObject(); + + for (Map.Entry entry : jsonObject.entrySet()) { + String attributeName = entry.getKey(); + JsonElement attributeValue = entry.getValue(); + + if (DynamoDBTypeConstants.BINARY.equalsIgnoreCase(attributeName)) { + return AttributeValue.fromB(context.deserialize(attributeValue, SdkBytes.class)); + } + + if (DynamoDBTypeConstants.BINARY_SET.equalsIgnoreCase(attributeName)) { + JsonArray jsonArray = attributeValue.getAsJsonArray(); + if (!jsonArray.isEmpty()) { + List sdkBytesList = new ArrayList<>(); + jsonArray.forEach(item -> sdkBytesList.add(context.deserialize(item, SdkBytes.class))); + return AttributeValue.fromBs(sdkBytesList); + } + } + + if (DynamoDBTypeConstants.BOOLEAN.equalsIgnoreCase(attributeName)) { + return AttributeValue.fromBool(attributeValue.getAsBoolean()); + } + + if (DynamoDBTypeConstants.NULL.equalsIgnoreCase(attributeName)) { + return AttributeValue.fromNul(attributeValue.getAsBoolean()); + } + + if (DynamoDBTypeConstants.NUMBER.equalsIgnoreCase(attributeName)) { + return AttributeValue.fromN(attributeValue.getAsString()); + } + + if (DynamoDBTypeConstants.NUMBER_SET.equalsIgnoreCase(attributeName)) { + JsonArray jsonArray = attributeValue.getAsJsonArray(); + if (!jsonArray.isEmpty()) { + List numberList = new ArrayList<>(); + jsonArray.forEach(item -> numberList.add(item.getAsString())); + return AttributeValue.fromNs(numberList); + } + } + + if (DynamoDBTypeConstants.LIST.equalsIgnoreCase(attributeName)) { + JsonArray jsonArray = attributeValue.getAsJsonArray(); + if (!jsonArray.isEmpty()) { + List avl = new ArrayList<>(); + jsonArray.forEach(element -> + avl.add(context.deserialize(element, AttributeValue.class))); + return AttributeValue.fromL(avl); + } + } + + if (DynamoDBTypeConstants.MAP.equalsIgnoreCase(attributeName)) { + JsonObject jsonMap = attributeValue.getAsJsonObject(); + if (jsonMap.size() != 0) { + Map avm = new HashMap<>(); + jsonMap.entrySet().forEach(item -> { + avm.put(item.getKey(), context.deserialize(item.getValue(), AttributeValue.class)); + }); + return AttributeValue.fromM(avm); + } + } + + if (DynamoDBTypeConstants.STRING.equalsIgnoreCase(attributeName)) { + return AttributeValue.fromS(attributeValue.getAsString()); + } + + if (DynamoDBTypeConstants.STRING_SET.equalsIgnoreCase(attributeName)) { + JsonArray jsonArray = attributeValue.getAsJsonArray(); + if (!jsonArray.isEmpty()) { + List stringList = new ArrayList<>(); + jsonArray.forEach(item -> stringList.add(item.getAsString())); + return AttributeValue.fromSs(stringList); + } + } + } + } + + // Return an empty instance as default value. + return AttributeValue.builder().build(); + } +} diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueSerializer.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueSerializer.java new file mode 100644 index 00000000..d8e4a7ff --- /dev/null +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/AttributeValueSerializer.java @@ -0,0 +1,97 @@ +/** + * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). You may not use this file + * except in compliance with the License. A copy of the License is located at + * + *     http://aws.amazon.com/apache2.0/ + * + * or in the "LICENSE.TXT" file accompanying this file. This file 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.hadoop.dynamodb; + +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonNull; +import com.google.gson.JsonObject; +import com.google.gson.JsonPrimitive; +import com.google.gson.JsonSerializationContext; +import com.google.gson.JsonSerializer; +import java.lang.reflect.Type; +import org.apache.hadoop.dynamodb.type.DynamoDBTypeConstants; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; + +public class AttributeValueSerializer implements JsonSerializer { + + @Override + public JsonElement serialize(AttributeValue attributeValue, Type type, + JsonSerializationContext context) { + if (attributeValue == null) { + return JsonNull.INSTANCE; + } + + JsonObject serializedValue = new JsonObject(); + switch (attributeValue.type()) { + case B: + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.BINARY), + context.serialize(attributeValue.b())); + break; + case BOOL: + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.BOOLEAN), + new JsonPrimitive(attributeValue.bool())); + break; + case BS: + JsonArray sdkBytesList = new JsonArray(); + attributeValue.bs() + .forEach(item -> sdkBytesList.add(context.serialize(item, SdkBytes.class))); + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.BINARY_SET), sdkBytesList); + break; + case L: + JsonArray attributeList = new JsonArray(); + attributeValue.l() + .forEach(item -> attributeList.add(context.serialize(item, AttributeValue.class))); + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.LIST), attributeList); + break; + case M: + JsonObject avm = new JsonObject(); + attributeValue.m().entrySet() + .forEach(entry -> + avm.add(entry.getKey(), context.serialize(entry.getValue(), AttributeValue.class))); + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.MAP), avm); + break; + case N: + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.NUMBER), + new JsonPrimitive(attributeValue.n())); + break; + case NS: + JsonArray numberList = new JsonArray(); + attributeValue.ns().forEach(item -> numberList.add(new JsonPrimitive(item))); + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.NUMBER_SET), numberList); + break; + case NUL: + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.NULL), + new JsonPrimitive(attributeValue.nul())); + break; + case S: + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.STRING), + new JsonPrimitive(attributeValue.s())); + break; + case SS: + JsonArray stringList = new JsonArray(); + attributeValue.ss().forEach(item -> stringList.add(new JsonPrimitive(item))); + serializedValue.add(toV1FieldCasingStyle(DynamoDBTypeConstants.STRING_SET), stringList); + break; + default: + break; + } + return serializedValue; + } + + private static String toV1FieldCasingStyle(String typeConstant) { + return typeConstant.substring(0, 1).toLowerCase() + typeConstant.substring(1).toUpperCase(); + } +} diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBClient.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBClient.java index a5637670..a9d71d4c 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBClient.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBClient.java @@ -20,38 +20,14 @@ import static org.apache.hadoop.dynamodb.DynamoDBConstants.MAX_ITEMS_PER_BATCH; import static org.apache.hadoop.dynamodb.DynamoDBConstants.MAX_ITEM_SIZE; import static org.apache.hadoop.dynamodb.DynamoDBUtil.getDynamoDBEndpoint; +import static org.apache.hadoop.dynamodb.DynamoDBUtil.getDynamoDBRegion; -import com.amazonaws.AmazonClientException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSCredentialsProviderChain; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.auth.BasicSessionCredentials; -import com.amazonaws.auth.InstanceProfileCredentialsProvider; -import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.BatchWriteItemRequest; -import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult; -import com.amazonaws.services.dynamodbv2.model.Capacity; -import com.amazonaws.services.dynamodbv2.model.Condition; -import com.amazonaws.services.dynamodbv2.model.ConsumedCapacity; -import com.amazonaws.services.dynamodbv2.model.DeleteRequest; -import com.amazonaws.services.dynamodbv2.model.DescribeTableRequest; -import com.amazonaws.services.dynamodbv2.model.DescribeTableResult; -import com.amazonaws.services.dynamodbv2.model.PutRequest; -import com.amazonaws.services.dynamodbv2.model.QueryRequest; -import com.amazonaws.services.dynamodbv2.model.QueryResult; -import com.amazonaws.services.dynamodbv2.model.ReturnConsumedCapacity; -import com.amazonaws.services.dynamodbv2.model.ScanRequest; -import com.amazonaws.services.dynamodbv2.model.ScanResult; -import com.amazonaws.services.dynamodbv2.model.TableDescription; -import com.amazonaws.services.dynamodbv2.model.WriteRequest; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.primitives.Ints; import java.io.UnsupportedEncodingException; +import java.net.URI; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -72,6 +48,36 @@ import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.util.ReflectionUtils; import org.joda.time.Duration; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbClientBuilder; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemRequest; +import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemResponse; +import software.amazon.awssdk.services.dynamodb.model.Capacity; +import software.amazon.awssdk.services.dynamodb.model.Condition; +import software.amazon.awssdk.services.dynamodb.model.ConsumedCapacity; +import software.amazon.awssdk.services.dynamodb.model.DeleteRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.PutRequest; +import software.amazon.awssdk.services.dynamodb.model.QueryRequest; +import software.amazon.awssdk.services.dynamodb.model.QueryResponse; +import software.amazon.awssdk.services.dynamodb.model.ReturnConsumedCapacity; +import software.amazon.awssdk.services.dynamodb.model.ScanRequest; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; +import software.amazon.awssdk.services.dynamodb.model.WriteRequest; public class DynamoDBClient { @@ -85,18 +91,18 @@ public class DynamoDBClient { DynamoDBConstants.DYNAMODB_SECRET_KEY_CONF ); private static final CredentialPairName DYNAMODB_SESSION_CREDENTIAL_PAIR_NAME = - new CredentialPairName( - DYNAMODB_CREDENTIAL_PAIR_NAME.getAccessKeyName(), - DYNAMODB_CREDENTIAL_PAIR_NAME.getSecretKeyName(), - DynamoDBConstants.DYNAMODB_SESSION_TOKEN_CONF - ); + new CredentialPairName( + DYNAMODB_CREDENTIAL_PAIR_NAME.getAccessKeyName(), + DYNAMODB_CREDENTIAL_PAIR_NAME.getSecretKeyName(), + DynamoDBConstants.DYNAMODB_SESSION_TOKEN_CONF + ); private static final CredentialPairName DEFAULT_CREDENTIAL_PAIR_NAME = new CredentialPairName( DynamoDBConstants.DEFAULT_ACCESS_KEY_CONF, DynamoDBConstants.DEFAULT_SECRET_KEY_CONF ); private final Map> writeBatchMap = new HashMap<>(); - private final AmazonDynamoDBClient dynamoDB; + private final DynamoDbClient dynamoDB; private int writeBatchMapSizeBytes; private int batchWriteRetries; private final Configuration config; @@ -118,8 +124,7 @@ public DynamoDBClient(Configuration conf) { public DynamoDBClient(Configuration conf, String region) { Preconditions.checkNotNull(conf, "conf cannot be null."); config = conf; - dynamoDB = getDynamoDBClient(conf); - dynamoDB.setEndpoint(getDynamoDBEndpoint(conf, region)); + dynamoDB = getDynamoDBClient(conf, region); maxBatchSize = config.getLong(MAX_BATCH_SIZE, DEFAULT_MAX_BATCH_SIZE); maxItemByteSize = config.getLong(MAX_ITEM_SIZE, DEFAULT_MAX_ITEM_SIZE); } @@ -129,80 +134,77 @@ public final Map> getWriteBatchMap() { } public TableDescription describeTable(String tableName) { - final DescribeTableRequest describeTablesRequest = new DescribeTableRequest() - .withTableName(tableName); + final DescribeTableRequest describeTablesRequest = DescribeTableRequest.builder() + .tableName(tableName) + .build(); try { - RetryResult describeResult = getRetryDriver().runWithRetry( - new Callable() { - @Override - public DescribeTableResult call() { - DescribeTableResult result = dynamoDB.describeTable(describeTablesRequest); - log.info("Describe table output: " + result); - return result; - } + RetryResult describeResult = getRetryDriver().runWithRetry( + () -> { + DescribeTableResponse response = dynamoDB.describeTable(describeTablesRequest); + log.info("Describe table output: " + response); + return response; }, null, null); - return describeResult.result.getTable(); + return describeResult.result.table(); } catch (Exception e) { throw new RuntimeException("Could not lookup table " + tableName + " in DynamoDB.", e); } } - public RetryResult scanTable( + public RetryResult scanTable( String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { - final ScanRequest scanRequest = new ScanRequest(tableName) - .withExclusiveStartKey(exclusiveStartKey) - .withLimit(Ints.checkedCast(limit)) - .withSegment(segment) - .withTotalSegments(totalSegments) - .withReturnConsumedCapacity(ReturnConsumedCapacity.TOTAL); + final ScanRequest.Builder scanRequestBuilder = ScanRequest.builder().tableName(tableName) + .exclusiveStartKey(exclusiveStartKey) + .limit(Ints.checkedCast(limit)) + .segment(segment) + .totalSegments(totalSegments) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL); if (dynamoDBQueryFilter != null) { Map scanFilter = dynamoDBQueryFilter.getScanFilter(); if (!scanFilter.isEmpty()) { - scanRequest.setScanFilter(scanFilter); + scanRequestBuilder.scanFilter(scanFilter); } } - RetryResult retryResult = getRetryDriver().runWithRetry(new Callable() { - @Override - public ScanResult call() { - log.debug("Executing DynamoDB scan: " + scanRequest); - return dynamoDB.scan(scanRequest); - } + final ScanRequest scanRequest = scanRequestBuilder.build(); + + RetryResult retryResult = getRetryDriver().runWithRetry(() -> { + log.debug("Executing DynamoDB scan: " + scanRequest); + return dynamoDB.scan(scanRequest); }, reporter, PrintCounter.DynamoDBReadThrottle); return retryResult; } - public RetryResult queryTable( + public RetryResult queryTable( String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Map exclusiveStartKey, long limit, Reporter reporter) { - final QueryRequest queryRequest = new QueryRequest() - .withTableName(tableName) - .withExclusiveStartKey(exclusiveStartKey) - .withKeyConditions(dynamoDBQueryFilter.getKeyConditions()) - .withLimit(Ints.checkedCast(limit)) - .withReturnConsumedCapacity(ReturnConsumedCapacity.TOTAL); + final QueryRequest.Builder queryRequestBuilder = QueryRequest.builder() + .tableName(tableName) + .exclusiveStartKey(exclusiveStartKey) + .keyConditions(dynamoDBQueryFilter.getKeyConditions()) + .limit(Ints.checkedCast(limit)) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL); DynamoDBIndexInfo index = dynamoDBQueryFilter.getIndex(); if (index != null) { log.debug("Using DynamoDB index: " + index.getIndexName()); - queryRequest.setIndexName(index.getIndexName()); + queryRequestBuilder.indexName(index.getIndexName()); } - RetryResult retryResult = getRetryDriver().runWithRetry( - new Callable() { - @Override - public QueryResult call() { - log.debug("Executing DynamoDB query: " + queryRequest); - return dynamoDB.query(queryRequest); - } + final QueryRequest queryRequest = queryRequestBuilder.build(); + + RetryResult retryResult = getRetryDriver().runWithRetry( + () -> { + log.debug("Executing DynamoDB query: " + queryRequest); + return dynamoDB.query(queryRequest); }, reporter, PrintCounter.DynamoDBReadThrottle); return retryResult; } - public BatchWriteItemResult putBatch(String tableName, Map item, - long maxItemsPerBatch, Reporter reporter, boolean deletionMode) + public BatchWriteItemResponse putBatch(String tableName, Map item, + long maxItemsPerBatch, Reporter reporter, + boolean deletionMode) throws UnsupportedEncodingException { int itemSizeBytes = DynamoDBUtil.getItemSizeBytes(item); @@ -211,7 +213,7 @@ public BatchWriteItemResult putBatch(String tableName, Map maxBatchSize; if (writeRequestsForTableAtLimit || totalSizeOfWriteBatchesOverLimit) { - result = writeBatch(reporter, itemSizeBytes); + response = writeBatch(reporter, itemSizeBytes); } } // writeBatchMap could be cleared from writeBatch() @@ -236,15 +238,22 @@ public BatchWriteItemResult putBatch(String tableName, Map getKeys(final Map it /** * @param roomNeeded number of bytes that writeBatch MUST make room for */ - private BatchWriteItemResult writeBatch(Reporter reporter, final int roomNeeded) { - final BatchWriteItemRequest batchWriteItemRequest = new BatchWriteItemRequest() - .withRequestItems(writeBatchMap) - .withReturnConsumedCapacity(ReturnConsumedCapacity.INDEXES); - - RetryResult retryResult = getRetryDriver().runWithRetry( - new Callable() { + private BatchWriteItemResponse writeBatch(Reporter reporter, final int roomNeeded) { + final BatchWriteItemRequest batchWriteItemRequest = BatchWriteItemRequest.builder() + .requestItems(writeBatchMap) + .returnConsumedCapacity(ReturnConsumedCapacity.INDEXES) + .build(); + + RetryResult retryResult = getRetryDriver().runWithRetry( + new Callable() { @Override - public BatchWriteItemResult call() throws + public BatchWriteItemResponse call() throws UnsupportedEncodingException, InterruptedException { pauseExponentially(batchWriteRetries); - BatchWriteItemResult result = dynamoDB.batchWriteItem(batchWriteItemRequest); + BatchWriteItemResponse result = dynamoDB.batchWriteItem(batchWriteItemRequest); - Map> unprocessedItems = result.getUnprocessedItems(); + Map> unprocessedItems = result.unprocessedItems(); if (unprocessedItems == null || unprocessedItems.isEmpty()) { batchWriteRetries = 0; } else { @@ -309,7 +319,7 @@ public BatchWriteItemResult call() throws int batchSizeBytes = 0; for (WriteRequest request : unprocessedWriteRequests) { batchSizeBytes += DynamoDBUtil.getItemSizeBytes( - request.getPutRequest().getItem()); + request.putRequest().item()); } long maxItemsPerBatch = @@ -318,24 +328,26 @@ public BatchWriteItemResult call() throws if (unprocessedWriteRequests.size() >= maxItemsPerBatch || (maxBatchSize - batchSizeBytes) < roomNeeded) { - throw new AmazonClientException("Full list of write requests not processed"); + throw SdkException.builder() + .message("Full list of write requests not processed") + .build(); } } double consumed = 0.0; - for (ConsumedCapacity consumedCapacity : result.getConsumedCapacity()) { - consumed = consumedCapacity.getTable().getCapacityUnits(); - if (consumedCapacity.getLocalSecondaryIndexes() != null) { + for (ConsumedCapacity consumedCapacity : result.consumedCapacity()) { + consumed = consumedCapacity.table().capacityUnits(); + if (consumedCapacity.localSecondaryIndexes() != null) { for (Capacity lsiConsumedCapacity : - consumedCapacity.getLocalSecondaryIndexes().values()) { - consumed += lsiConsumedCapacity.getCapacityUnits(); + consumedCapacity.localSecondaryIndexes().values()) { + consumed += lsiConsumedCapacity.capacityUnits(); } } } int batchSize = 0; for (List writeRequests : - batchWriteItemRequest.getRequestItems().values()) { + batchWriteItemRequest.requestItems().values()) { batchSize += writeRequests.size(); } @@ -352,12 +364,12 @@ public BatchWriteItemResult call() throws writeBatchMapSizeBytes = 0; // If some items failed to go through, add them back to the writeBatchMap - Map> unprocessedItems = retryResult.result.getUnprocessedItems(); + Map> unprocessedItems = retryResult.result.unprocessedItems(); for (Entry> entry : unprocessedItems.entrySet()) { String key = entry.getKey(); List requests = entry.getValue(); for (WriteRequest request : requests) { - writeBatchMapSizeBytes += DynamoDBUtil.getItemSizeBytes(request.getPutRequest().getItem()); + writeBatchMapSizeBytes += DynamoDBUtil.getItemSizeBytes(request.putRequest().item()); } writeBatchMap.put(key, requests); } @@ -379,22 +391,37 @@ private void pauseExponentially(int retries) throws InterruptedException { Thread.sleep(delay); } - private AmazonDynamoDBClient getDynamoDBClient(Configuration conf) { - ClientConfiguration clientConfig = new ClientConfiguration().withMaxErrorRetry(1); - applyProxyConfiguration(clientConfig, conf); - return new AmazonDynamoDBClient(getAWSCredentialsProvider(conf), clientConfig); + private DynamoDbClient getDynamoDBClient(Configuration conf, String region) { + final DynamoDbClientBuilder dynamoDbClientBuilder = DynamoDbClient.builder(); + + dynamoDbClientBuilder.region(Region.of(getDynamoDBRegion(conf, region))); + + String customEndpoint = getDynamoDBEndpoint(conf, region); + if (!Strings.isNullOrEmpty(customEndpoint)) { + dynamoDbClientBuilder.endpointOverride(URI.create(customEndpoint)); + } + + return dynamoDbClientBuilder.httpClient(ApacheHttpClient.builder() + .proxyConfiguration(applyProxyConfiguration(conf)) + .build()) + .credentialsProvider(getAwsCredentialsProvider(conf)) + .overrideConfiguration(ClientOverrideConfiguration.builder() + .retryPolicy(builder -> builder.numRetries(1)) + .build()) + .build(); } @VisibleForTesting - void applyProxyConfiguration(ClientConfiguration clientConfig, Configuration conf) { + ProxyConfiguration applyProxyConfiguration(Configuration conf) { + ProxyConfiguration.Builder builder = ProxyConfiguration.builder(); + final String proxyHost = conf.get(DynamoDBConstants.PROXY_HOST); final int proxyPort = conf.getInt(DynamoDBConstants.PROXY_PORT, 0); final String proxyUsername = conf.get(DynamoDBConstants.PROXY_USERNAME); final String proxyPassword = conf.get(DynamoDBConstants.PROXY_PASSWORD); boolean proxyHostAndPortPresent = false; if (!Strings.isNullOrEmpty(proxyHost) && proxyPort > 0) { - clientConfig.setProxyHost(proxyHost); - clientConfig.setProxyPort(proxyPort); + builder.endpoint(buildProxyEndpoint(proxyHost, proxyPort)); proxyHostAndPortPresent = true; } else if (Strings.isNullOrEmpty(proxyHost) ^ proxyPort <= 0) { throw new RuntimeException("Only one of proxy host and port are set, when both are required"); @@ -404,17 +431,19 @@ void applyProxyConfiguration(ClientConfiguration clientConfig, Configuration con throw new RuntimeException("Proxy host and port must be supplied if proxy username and " + "password are present"); } else { - clientConfig.setProxyUsername(proxyUsername); - clientConfig.setProxyPassword(proxyPassword); + builder.username(proxyUsername) + .password(proxyPassword); } } else if (Strings.isNullOrEmpty(proxyUsername) ^ Strings.isNullOrEmpty(proxyPassword)) { throw new RuntimeException("Only one of proxy username and password are set, when both are " + "required"); } + + return builder.build(); } - protected AWSCredentialsProvider getAWSCredentialsProvider(Configuration conf) { - List providersList = new ArrayList<>(); + protected AwsCredentialsProvider getAwsCredentialsProvider(Configuration conf) { + List providersList = new ArrayList<>(); // try to load custom credential provider, fail if a provider is specified but cannot be // initialized @@ -422,7 +451,7 @@ protected AWSCredentialsProvider getAWSCredentialsProvider(Configuration conf) { if (!Strings.isNullOrEmpty(providerClass)) { try { providersList.add( - (AWSCredentialsProvider) ReflectionUtils.newInstance(Class.forName(providerClass), conf) + (AwsCredentialsProvider) ReflectionUtils.newInstance(Class.forName(providerClass), conf) ); } catch (ClassNotFoundException e) { throw new RuntimeException("Custom AWSCredentialsProvider not found: " + providerClass, e); @@ -443,41 +472,31 @@ protected AWSCredentialsProvider getAWSCredentialsProvider(Configuration conf) { } if (Strings.isNullOrEmpty(accessKey) || Strings.isNullOrEmpty(secretKey)) { - providersList.add(new InstanceProfileCredentialsProvider()); + providersList.add(InstanceProfileCredentialsProvider.create()); } else if (!Strings.isNullOrEmpty(sessionKey)) { - final AWSCredentials credentials = - new BasicSessionCredentials(accessKey, secretKey, sessionKey); - providersList.add(new AWSCredentialsProvider() { - @Override - public AWSCredentials getCredentials() { - return credentials; - } - - @Override - public void refresh() { - } - }); + final AwsCredentials credentials = + AwsSessionCredentials.create(accessKey, secretKey, sessionKey); + providersList.add(() -> credentials); } else { - final AWSCredentials credentials = new BasicAWSCredentials(accessKey, secretKey); - providersList.add(new AWSCredentialsProvider() { - @Override - public AWSCredentials getCredentials() { - return credentials; - } - - @Override - public void refresh() { - } - }); + final AwsCredentials credentials = AwsBasicCredentials.create(accessKey, secretKey); + providersList.add(() -> credentials); } - AWSCredentialsProvider[] providerArray = providersList.toArray( - new AWSCredentialsProvider[providersList.size()] + AwsCredentialsProvider[] providerArray = providersList.toArray( + new AwsCredentialsProvider[providersList.size()] ); - AWSCredentialsProviderChain providerChain = new AWSCredentialsProviderChain(providerArray); - providerChain.setReuseLastProvider(true); + AwsCredentialsProviderChain providerChain = AwsCredentialsProviderChain.builder() + .credentialsProviders(providerArray) + .reuseLastProviderEnabled(true) + .build(); return providerChain; } + private URI buildProxyEndpoint(String proxyHost, int proxyPort) { + // Default proxy protocol is HTTP, aligning with AWS Java SDK 1.x + // https://github.com/aws/aws-sdk-java/blob/master/aws-java-sdk-core/src/main/java/com/amazonaws/ClientConfiguration.java#L171 + final String HTTP_PROTOCOL = "http://"; + return URI.create(HTTP_PROTOCOL + proxyHost + ":" + proxyPort); + } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBConstants.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBConstants.java index 89eae115..4d951913 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBConstants.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBConstants.java @@ -13,8 +13,8 @@ package org.apache.hadoop.dynamodb; -import com.amazonaws.regions.Regions; -import com.amazonaws.services.dynamodbv2.model.BillingMode; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; /** * Contains constants used for the Hadoop to DynamoDB connection. Note that many of these string @@ -104,7 +104,7 @@ public interface DynamoDBConstants { int RATE_CONTROLLER_WINDOW_SIZE_SEC = 5; String EXPORT_FORMAT_VERSION = "dynamodb.export.format.version"; - String DEFAULT_AWS_REGION = Regions.US_EAST_1.getName(); + String DEFAULT_AWS_REGION = Region.US_EAST_1.toString(); int DEFAULT_AVERAGE_ITEM_SIZE_IN_BYTES = 100; Long DEFAULT_CAPACITY_FOR_ON_DEMAND = 40000L; diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryer.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryer.java index 5fc18f1e..e294da22 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryer.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryer.java @@ -13,8 +13,6 @@ package org.apache.hadoop.dynamodb; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; import java.net.SocketException; import java.net.SocketTimeoutException; import java.util.HashSet; @@ -27,6 +25,8 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Duration; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; /** * FIXME This class is not thread safe. @@ -91,16 +91,17 @@ public synchronized void shutdown() { } private void handleException(DateTime retryEndTime, Exception exception, Reporter reporter, - PrintCounter retryCounter) { + PrintCounter retryCounter) { DateTime currentTime = new DateTime(DateTimeZone.UTC); long maxDelay = retryEndTime.getMillis() - currentTime.getMillis(); if (verifyRetriableException(exception) && maxDelay > 0) { - if (exception instanceof AmazonServiceException) { - AmazonServiceException ase = (AmazonServiceException) exception; - if (throttleErrorCodes.contains(ase.getErrorCode())) { + if (exception instanceof AwsServiceException) { + AwsServiceException ase = (AwsServiceException) exception; + if (throttleErrorCodes.contains(ase.awsErrorDetails().errorCode())) { // Retry exception - } else if (internalErrorStatusCodes.contains(ase.getStatusCode())) { + } else if (internalErrorStatusCodes.contains( + ase.awsErrorDetails().sdkHttpResponse().statusCode())) { // Retry exception } else { throw new RuntimeException(exception); @@ -121,8 +122,8 @@ private void handleException(DateTime retryEndTime, Exception exception, Reporte } private boolean verifyRetriableException(Exception exception) { - return exception instanceof AmazonServiceException - || exception instanceof AmazonClientException + return exception instanceof AwsServiceException + || exception instanceof SdkException || exception instanceof SocketException || exception instanceof SocketTimeoutException; } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBItemWritable.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBItemWritable.java index 6f3a4dfd..90a7ebc8 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBItemWritable.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBItemWritable.java @@ -13,7 +13,6 @@ package org.apache.hadoop.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import java.io.ByteArrayInputStream; @@ -27,6 +26,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.hadoop.io.Writable; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBItemWritable implements Writable, Serializable { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBUtil.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBUtil.java index 37c2a451..e26fcb01 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBUtil.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/DynamoDBUtil.java @@ -17,11 +17,6 @@ import static org.apache.hadoop.dynamodb.DynamoDBConstants.DEFAULT_SEGMENT_SPLIT_SIZE; import static org.apache.hadoop.dynamodb.DynamoDBConstants.MAX_ITEMS_PER_BATCH; -import com.amazonaws.regions.RegionUtils; -import com.amazonaws.regions.ServiceAbbreviations; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.TableDescription; -import com.amazonaws.util.EC2MetadataUtils; import com.google.common.base.Strings; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -54,6 +49,10 @@ import org.apache.hadoop.mapred.JobConf; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.regions.internal.util.EC2MetadataUtils; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public final class DynamoDBUtil { @@ -66,13 +65,17 @@ public final class DynamoDBUtil { /* We hand serialize/deserialize ByteBuffer objects. */ gsonBuilder.registerTypeAdapter(ByteBuffer.class, new ByteBufferSerializer()); gsonBuilder.registerTypeAdapter(ByteBuffer.class, new ByteBufferDeserializer()); + gsonBuilder.registerTypeAdapter(SdkBytes.class, new SdkBytesSerializer()); + gsonBuilder.registerTypeAdapter(SdkBytes.class, new SdkBytesDeserializer()); + gsonBuilder.registerTypeAdapter(AttributeValue.class, new AttributeValueSerializer()); + gsonBuilder.registerTypeAdapter(AttributeValue.class, new AttributeValueDeserializer()); gson = gsonBuilder.disableHtmlEscaping().create(); } public static Double calculateAverageItemSize(TableDescription description) { - if (description.getItemCount() != 0) { - return ((double) description.getTableSizeBytes()) / ((double) description.getItemCount()); + if (description.itemCount() != 0) { + return ((double) description.tableSizeBytes()) / ((double) description.itemCount()); } return 0.0; } @@ -166,31 +169,31 @@ public static void verifyInterval(long intervalBeginTime, long intervalLength) { private static int getAttributeSizeBytes(AttributeValue att) throws UnsupportedEncodingException { int byteSize = 0; - if (att.getN() != null) { - byteSize += att.getN().getBytes(CHARACTER_ENCODING).length; - } else if (att.getS() != null) { - byteSize += att.getS().getBytes(CHARACTER_ENCODING).length; - } else if (att.getB() != null) { - byteSize += att.getB().array().length; - } else if (att.getNS() != null) { - for (String number : att.getNS()) { + if (att.n() != null) { + byteSize += att.n().getBytes(CHARACTER_ENCODING).length; + } else if (att.s() != null) { + byteSize += att.s().getBytes(CHARACTER_ENCODING).length; + } else if (att.b() != null) { + byteSize += att.b().asByteArray().length; + } else if (att.hasNs()) { + for (String number : att.ns()) { byteSize += number.getBytes(CHARACTER_ENCODING).length; } - } else if (att.getSS() != null) { - for (String string : att.getSS()) { + } else if (att.hasSs()) { + for (String string : att.ss()) { byteSize += string.getBytes(CHARACTER_ENCODING).length; } - } else if (att.getBS() != null) { - for (ByteBuffer byteBuffer : att.getBS()) { - byteSize += byteBuffer.array().length; + } else if (att.hasBs()) { + for (SdkBytes sdkBytes : att.bs()) { + byteSize += sdkBytes.asByteArray().length; } - } else if (att.getM() != null) { - for (Entry entry : att.getM().entrySet()) { + } else if (att.hasM()) { + for (Entry entry : att.m().entrySet()) { byteSize += getAttributeSizeBytes(entry.getValue()) + entry.getKey().getBytes(CHARACTER_ENCODING).length; } - } else if (att.getL() != null) { - for (AttributeValue entry : att.getL()) { + } else if (att.hasL()) { + for (AttributeValue entry : att.l()) { byteSize += getAttributeSizeBytes(entry); } } @@ -214,17 +217,7 @@ public static String getValueFromConf(Configuration conf, String confKey) { } /** - * Calculates DynamoDB end-point. - * - * Algorithm details: - *
    - *
  1. Use endpoint in job configuration "dynamodb.endpoint" value if available - *
  2. Use endpoint from region in job configuration "dynamodb.region" value if available - *
  3. Use endpoint from region in job configuration "dynamodb.regionid" value if available - *
  4. Use endpoint from EC2 Metadata of instance if available - *
  5. If all previous attempts at retrieving endpoint fail, default to us-east-1 endpoint - *
- * + * Get custom DynamoDB end-point from configuration. * @param conf Job Configuration * @param region optional preferred region * @return end-point for DynamoDb service @@ -232,29 +225,60 @@ public static String getValueFromConf(Configuration conf, String confKey) { public static String getDynamoDBEndpoint(Configuration conf, String region) { String endpoint = getValueFromConf(conf, DynamoDBConstants.ENDPOINT); if (Strings.isNullOrEmpty(endpoint)) { - if (Strings.isNullOrEmpty(region)) { - region = getValueFromConf(conf, DynamoDBConstants.REGION); - } - if (Strings.isNullOrEmpty(region)) { - region = getValueFromConf(conf, DynamoDBConstants.REGION_ID); - } - if (Strings.isNullOrEmpty(region)) { - try { - region = EC2MetadataUtils.getEC2InstanceRegion(); - } catch (Exception e) { - log.warn(String.format("Exception when attempting to get AWS region information. Will " - + "ignore and default " + "to %s", DynamoDBConstants.DEFAULT_AWS_REGION), e); - } - } - if (Strings.isNullOrEmpty(region)) { - region = DynamoDBConstants.DEFAULT_AWS_REGION; - } - endpoint = RegionUtils.getRegion(region).getServiceEndpoint(ServiceAbbreviations.Dynamodb); + log.info(DynamoDBConstants.ENDPOINT + " not found from configuration."); + } else { + log.info("Using endpoint for DynamoDB: " + endpoint); } - log.info("Using endpoint for DynamoDB: " + endpoint); return endpoint; } + /** + * Calculates DynamoDB region. + * + * Algorithm details: + *
    + *
  1. Use region in job configuration "dynamodb.region" value if available + *
  2. Use region in job configuration "dynamodb.regionid" value if available + *
  3. Use endpoint from EC2 Metadata of instance if available + *
  4. If all previous attempts fail, default to us-east-1 region + *
+ * + * @param conf + * @param region + * @return region for DynamoDB service + */ + public static String getDynamoDBRegion(Configuration conf, String region) { + if (!Strings.isNullOrEmpty(region)) { + return region; + } + + // Return region in job configuration "dynamodb.region" value if available + region = getValueFromConf(conf, DynamoDBConstants.REGION); + if (!Strings.isNullOrEmpty(region)) { + return region; + } + + // Return region in job configuration "dynamodb.regionid" value if available + region = getValueFromConf(conf, DynamoDBConstants.REGION_ID); + if (!Strings.isNullOrEmpty(region)) { + return region; + } + + // Return region from EC2 Metadata of instance if available + try { + region = EC2MetadataUtils.getEC2InstanceRegion(); + } catch (Exception e) { + log.warn(String.format("Exception when attempting to get AWS region information. " + + "Will ignore and default to %s", DynamoDBConstants.DEFAULT_AWS_REGION), e); + } + if (!Strings.isNullOrEmpty(region)) { + return region; + } + + // Default to us-east-1 region if all previous attempts fail + return DynamoDBConstants.DEFAULT_AWS_REGION; + } + public static JobClient createJobClient(JobConf jobConf) { try { return new JobClient(jobConf); @@ -336,6 +360,28 @@ public ByteBuffer deserialize(JsonElement jsonElement, Type type, JsonDeserializ } } + private static class SdkBytesSerializer implements JsonSerializer { + + @Override + public JsonElement serialize(SdkBytes sdkBytes, Type type, JsonSerializationContext + context) { + + String base64String = DynamoDBUtil.base64EncodeByteArray(sdkBytes.asByteArray()); + return new JsonPrimitive(base64String); + } + } + + private static class SdkBytesDeserializer implements JsonDeserializer { + + @Override + public SdkBytes deserialize(JsonElement jsonElement, Type type, JsonDeserializationContext + context) throws JsonParseException { + + String base64String = jsonElement.getAsJsonPrimitive().getAsString(); + return SdkBytes.fromByteBuffer(DynamoDBUtil.base64StringToByteBuffer(base64String)); + } + } + private DynamoDBUtil() { } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBFilter.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBFilter.java index 006405ad..9b8337e7 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBFilter.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBFilter.java @@ -13,7 +13,7 @@ package org.apache.hadoop.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.Condition; +import software.amazon.awssdk.services.dynamodb.model.Condition; public interface DynamoDBFilter { String getColumnName(); diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBIndexInfo.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBIndexInfo.java index b2fa97ec..93811228 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBIndexInfo.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBIndexInfo.java @@ -1,8 +1,8 @@ package org.apache.hadoop.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.Projection; import java.util.List; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.Projection; public class DynamoDBIndexInfo { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBQueryFilter.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBQueryFilter.java index 246e4fdb..e780ae17 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBQueryFilter.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/filter/DynamoDBQueryFilter.java @@ -13,9 +13,9 @@ package org.apache.hadoop.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.Condition; import java.util.HashMap; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.Condition; public class DynamoDBQueryFilter { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBinaryKey.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBinaryKey.java index 6f472df2..07dad545 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBinaryKey.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBinaryKey.java @@ -13,9 +13,9 @@ package org.apache.hadoop.dynamodb.key; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.nio.ByteBuffer; import org.apache.hadoop.dynamodb.DynamoDBUtil; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBBinaryKey extends AbstractDynamoDBKey { @@ -28,6 +28,6 @@ public DynamoDBBinaryKey(String base64EncodedKey) { @Override public int compareValue(AttributeValue attribute) { - return byteBuffer.compareTo(attribute.getB()); + return byteBuffer.compareTo(attribute.b().asByteBuffer()); } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBooleanKey.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBooleanKey.java index c09f604d..51e91bc3 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBooleanKey.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBBooleanKey.java @@ -13,7 +13,7 @@ package org.apache.hadoop.dynamodb.key; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBBooleanKey extends AbstractDynamoDBKey { @@ -24,7 +24,7 @@ public DynamoDBBooleanKey(String key) { @Override public int compareValue(AttributeValue attribute) { - return new Boolean(key).compareTo(attribute.getBOOL()); + return new Boolean(key).compareTo(attribute.bool()); } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBKey.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBKey.java index a00ce89c..a6dd4463 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBKey.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBKey.java @@ -13,7 +13,7 @@ package org.apache.hadoop.dynamodb.key; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public interface DynamoDBKey { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBNumberKey.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBNumberKey.java index 9bfe9cb5..0eb89bd9 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBNumberKey.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBNumberKey.java @@ -13,8 +13,8 @@ package org.apache.hadoop.dynamodb.key; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.math.BigDecimal; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBNumberKey extends AbstractDynamoDBKey { @@ -26,7 +26,7 @@ public DynamoDBNumberKey(String key) { @Override public int compareValue(AttributeValue attribute) { - BigDecimal passedKey = new BigDecimal(attribute.getN()); + BigDecimal passedKey = new BigDecimal(attribute.n()); if (number == null) { number = new BigDecimal(key); } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBStringKey.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBStringKey.java index 4f797277..5cc3bc2b 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBStringKey.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/key/DynamoDBStringKey.java @@ -13,7 +13,7 @@ package org.apache.hadoop.dynamodb.key; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBStringKey extends AbstractDynamoDBKey { @@ -23,6 +23,6 @@ public DynamoDBStringKey(String key) { @Override public int compareValue(AttributeValue attribute) { - return key.compareTo(attribute.getS()); + return key.compareTo(attribute.s()); } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractReadManager.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractReadManager.java index 98e81874..b899f14e 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractReadManager.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractReadManager.java @@ -13,7 +13,6 @@ package org.apache.hadoop.dynamodb.preader; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.ArrayList; import java.util.Deque; import java.util.List; @@ -26,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.dynamodb.util.AbstractTimeSource; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; /** * The ReadManager is responsible for deciding the required number of ReadWorkers to achieve the diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractRecordReadRequest.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractRecordReadRequest.java index 813b1173..23e8689c 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractRecordReadRequest.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/AbstractRecordReadRequest.java @@ -13,10 +13,10 @@ package org.apache.hadoop.dynamodb.preader; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.Map; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.dynamodb.preader.RateController.RequestLimit; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public abstract class AbstractRecordReadRequest { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/DynamoDBRecordReaderContext.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/DynamoDBRecordReaderContext.java index 2c0847ca..9cd7be54 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/DynamoDBRecordReaderContext.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/DynamoDBRecordReaderContext.java @@ -13,7 +13,6 @@ package org.apache.hadoop.dynamodb.preader; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.Collection; import java.util.Map; import org.apache.hadoop.dynamodb.DynamoDBClient; @@ -21,6 +20,7 @@ import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reporter; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBRecordReaderContext { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/QueryRecordReadRequest.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/QueryRecordReadRequest.java index f3585678..5bcb4f58 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/QueryRecordReadRequest.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/QueryRecordReadRequest.java @@ -1,11 +1,11 @@ /** * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * + *

* Licensed under the Apache License, Version 2.0 (the "License"). You may not use this file * except in compliance with the License. A copy of the License is located at - * + *

*     http://aws.amazon.com/apache2.0/ - * + *

* or in the "LICENSE.TXT" file accompanying this file. This file 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. @@ -13,35 +13,40 @@ package org.apache.hadoop.dynamodb.preader; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.QueryResult; import java.util.Map; import org.apache.hadoop.dynamodb.DynamoDBFibonacciRetryer.RetryResult; import org.apache.hadoop.dynamodb.preader.RateController.RequestLimit; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.QueryResponse; public class QueryRecordReadRequest extends AbstractRecordReadRequest { public QueryRecordReadRequest(AbstractReadManager readMgr, DynamoDBRecordReaderContext context, - Map lastEvaluatedKey) { + Map lastEvaluatedKey) { super(readMgr, context, 0 /* segment */, lastEvaluatedKey); } @Override protected AbstractRecordReadRequest buildNextReadRequest(PageResults> pageResults) { + AttributeValue>> pageResults) { return new QueryRecordReadRequest(readMgr, context, pageResults.lastEvaluatedKey); } @Override protected PageResults> fetchPage(RequestLimit lim) { // Read from DynamoDB - RetryResult retryResult = context.getClient().queryTable(tableName, context - .getSplit().getFilterPushdown(), lastEvaluatedKey, lim.items, context.getReporter()); + RetryResult retryResult = context.getClient().queryTable(tableName, context + .getSplit().getFilterPushdown(), lastEvaluatedKey, lim.items, context.getReporter()); - QueryResult result = retryResult.result; + QueryResponse response = retryResult.result; int retries = retryResult.retries; - return new PageResults<>(result.getItems(), result.getLastEvaluatedKey(), result - .getConsumedCapacity().getCapacityUnits(), retries); + return new PageResults<>(response.items(), + // Default value of QueryResponse.lastEvaluatedKey is changed from NULL to + // SdkAutoConstructMap in AWS SDK 2.x. + // Translate the default value to NULL here, to keep this assumption in other classes. + response.hasLastEvaluatedKey() ? response.lastEvaluatedKey() : null, + response.consumedCapacity().capacityUnits(), + retries); } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequest.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequest.java index fd7f3afa..89fb8ded 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequest.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequest.java @@ -13,11 +13,11 @@ package org.apache.hadoop.dynamodb.preader; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.ScanResult; import java.util.Map; import org.apache.hadoop.dynamodb.DynamoDBFibonacciRetryer.RetryResult; import org.apache.hadoop.dynamodb.preader.RateController.RequestLimit; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; public class ScanRecordReadRequest extends AbstractRecordReadRequest { @@ -35,17 +35,23 @@ protected AbstractRecordReadRequest buildNextReadRequest(PageResults> fetchPage(RequestLimit lim) { // Read from DynamoDB - RetryResult retryResult = context.getClient().scanTable(tableName, null, segment, - context.getSplit().getTotalSegments(), lastEvaluatedKey, lim.items, context.getReporter()); + RetryResult retryResult = context.getClient() + .scanTable(tableName, null, segment, context.getSplit().getTotalSegments(), + lastEvaluatedKey, lim.items, context.getReporter()); - ScanResult result = retryResult.result; + ScanResponse response = retryResult.result; int retries = retryResult.retries; double consumedCapacityUnits = 0.0; - if (result.getConsumedCapacity() != null) { - consumedCapacityUnits = result.getConsumedCapacity().getCapacityUnits(); + if (response.consumedCapacity() != null) { + consumedCapacityUnits = response.consumedCapacity().capacityUnits(); } - return new PageResults<>(result.getItems(), result.getLastEvaluatedKey(), consumedCapacityUnits, + return new PageResults<>(response.items(), + // Default value of ScanResponse.lastEvaluatedKey is changed from NULL to + // SdkAutoConstructMap in AWS SDK 2.x. + // Translate the default value to NULL here, to keep this assumption in other classes. + response.hasLastEvaluatedKey() ? response.lastEvaluatedKey() : null, + consumedCapacityUnits, retries); } } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/AbstractDynamoDBRecordReader.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/AbstractDynamoDBRecordReader.java index 0d808246..25ede33d 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/AbstractDynamoDBRecordReader.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/AbstractDynamoDBRecordReader.java @@ -15,7 +15,6 @@ import static org.apache.hadoop.dynamodb.DynamoDBUtil.createJobClient; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.io.IOException; import java.util.Map; import org.apache.commons.logging.Log; @@ -34,6 +33,7 @@ import org.apache.hadoop.dynamodb.util.TimeSource; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; /** * AbstractDynamoDBRecordReader does all the backend work for splitting up the data in DynamoDB, diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculator.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculator.java index 82ac3b6d..7634d965 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculator.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculator.java @@ -13,8 +13,6 @@ package org.apache.hadoop.dynamodb.read; -import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dynamodb.DynamoDBClient; @@ -22,6 +20,9 @@ import org.apache.hadoop.dynamodb.IopsCalculator; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputDescription; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class ReadIopsCalculator implements IopsCalculator { @@ -66,12 +67,11 @@ public long calculateTargetIops() { private double getThroughput() { TableDescription tableDescription = dynamoDBClient.describeTable(tableName); - if (tableDescription.getBillingModeSummary() == null - || tableDescription.getBillingModeSummary().getBillingMode() - .equalsIgnoreCase(DynamoDBConstants.BILLING_MODE_PROVISIONED)) { + if (tableDescription.billingModeSummary() == null + || tableDescription.billingModeSummary().billingMode() == BillingMode.PROVISIONED) { ProvisionedThroughputDescription provisionedThroughput = tableDescription - .getProvisionedThroughput(); - return provisionedThroughput.getReadCapacityUnits(); + .provisionedThroughput(); + return provisionedThroughput.readCapacityUnits(); } return DynamoDBConstants.DEFAULT_CAPACITY_FOR_ON_DEMAND; } diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinarySetType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinarySetType.java index 74bf7bd7..74389d2a 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinarySetType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinarySetType.java @@ -13,15 +13,20 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import java.util.stream.Collectors; import org.apache.hadoop.dynamodb.DynamoDBUtil; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBBinarySetType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withBS(DynamoDBUtil.base64StringToByteBuffer(values)); + return AttributeValue.fromBs(DynamoDBUtil.base64StringToByteBuffer(values) + .stream() + .map(byteBuffer -> SdkBytes.fromByteBuffer(byteBuffer)) + .collect(Collectors.toList())); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinaryType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinaryType.java index 61f86488..dc5e9968 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinaryType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBinaryType.java @@ -15,15 +15,16 @@ import static org.apache.hadoop.dynamodb.DynamoDBUtil.base64StringToByteBuffer; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBBinaryKey; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBBinaryType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withB(base64StringToByteBuffer(values[0])); + return AttributeValue.fromB(SdkBytes.fromByteBuffer(base64StringToByteBuffer(values[0]))); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBooleanType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBooleanType.java index 633ee55a..54773bdc 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBooleanType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBBooleanType.java @@ -13,16 +13,16 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBBooleanKey; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBBooleanType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withBOOL(new Boolean(values[0])); + return AttributeValue.fromBool(new Boolean(values[0])); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBListType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBListType.java index d60de6d9..4deea40e 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBListType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBListType.java @@ -11,18 +11,19 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import java.util.ArrayList; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBListType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - AttributeValue av = new AttributeValue(); + ArrayList avl = new ArrayList<>(); for (String ele : values) { - av.withL(new AttributeValue(ele)); + avl.add(AttributeValue.fromS(ele)); } - return av; + return AttributeValue.fromL(avl); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBMapType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBMapType.java index e14d8a6e..f6fcf138 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBMapType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBMapType.java @@ -11,8 +11,8 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBMapType implements DynamoDBType { @@ -21,7 +21,7 @@ public AttributeValue getAttributeValue(String... values) { System.out.println("values:"); System.out.println(values); System.out.println("end values"); - return new AttributeValue();//.withM(values[0]); + return AttributeValue.builder().build(); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNullType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNullType.java index 91f0a29a..4a420294 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNullType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNullType.java @@ -1,12 +1,12 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBNullType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withNULL(true); + return AttributeValue.fromNul(true); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberSetType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberSetType.java index b05fe7c7..3880dd3c 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberSetType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberSetType.java @@ -13,14 +13,15 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import java.util.Arrays; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBNumberSetType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withNS(values); + return AttributeValue.fromNs(Arrays.asList(values)); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberType.java index 33a3a46f..2e305032 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBNumberType.java @@ -13,15 +13,15 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBKey; import org.apache.hadoop.dynamodb.key.DynamoDBNumberKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBNumberType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withN(values[0]); + return AttributeValue.fromN(values[0]); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringSetType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringSetType.java index b094d1a0..9f4f1a7e 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringSetType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringSetType.java @@ -13,14 +13,15 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; +import java.util.Arrays; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBStringSetType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue().withSS(values); + return AttributeValue.fromSs(Arrays.asList(values)); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringType.java index 4dcebf10..0fdb551b 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBStringType.java @@ -13,15 +13,15 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBKey; import org.apache.hadoop.dynamodb.key.DynamoDBStringKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBStringType implements DynamoDBType { @Override public AttributeValue getAttributeValue(String... values) { - return new AttributeValue(values[0]); + return AttributeValue.fromS(values[0]); } @Override diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBType.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBType.java index 2f3b67c6..0aaf57e7 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBType.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/type/DynamoDBType.java @@ -13,8 +13,8 @@ package org.apache.hadoop.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.key.DynamoDBKey; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public interface DynamoDBType { diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/AbstractDynamoDBRecordWriter.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/AbstractDynamoDBRecordWriter.java index 52386284..59a02875 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/AbstractDynamoDBRecordWriter.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/AbstractDynamoDBRecordWriter.java @@ -16,11 +16,6 @@ import static org.apache.hadoop.dynamodb.DynamoDBConstants.DEFAULT_AVERAGE_ITEM_SIZE_IN_BYTES; import static org.apache.hadoop.dynamodb.DynamoDBUtil.createJobClient; -import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult; -import com.amazonaws.services.dynamodbv2.model.Capacity; -import com.amazonaws.services.dynamodbv2.model.ConsumedCapacity; -import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; -import com.amazonaws.services.dynamodbv2.model.WriteRequest; import java.io.IOException; import java.util.List; import org.apache.commons.logging.Log; @@ -38,6 +33,11 @@ import org.apache.hadoop.util.Progressable; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemResponse; +import software.amazon.awssdk.services.dynamodb.model.Capacity; +import software.amazon.awssdk.services.dynamodb.model.ConsumedCapacity; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.WriteRequest; /** * AbstractDynamoDBRecordWriter does all the backend work for reading in key-value pairs from the @@ -75,7 +75,9 @@ public AbstractDynamoDBRecordWriter(JobConf jobConf, Progressable progressable) client = new DynamoDBClient(jobConf); tableName = jobConf.get(DynamoDBConstants.OUTPUT_TABLE_NAME); if (tableName == null) { - throw new ResourceNotFoundException("No output table name was specified."); + throw ResourceNotFoundException.builder() + .message("No output table name was specified.") + .build(); } @@ -109,20 +111,20 @@ public void write(K key, V value) throws IOException { } DynamoDBItemWritable item = convertValueToDynamoDBItem(key, value); - BatchWriteItemResult result = client.putBatch(tableName, item.getItem(), + BatchWriteItemResponse response = client.putBatch(tableName, item.getItem(), permissibleWritesPerSecond - writesPerSecond, reporter, deletionMode); batchSize++; totalItemsWritten++; - if (result != null) { - if (result.getConsumedCapacity() != null) { - for (ConsumedCapacity consumedCapacity : result.getConsumedCapacity()) { - double consumedUnits = consumedCapacity.getTable().getCapacityUnits(); - if (consumedCapacity.getLocalSecondaryIndexes() != null) { + if (response != null) { + if (response.consumedCapacity() != null) { + for (ConsumedCapacity consumedCapacity : response.consumedCapacity()) { + double consumedUnits = consumedCapacity.table().capacityUnits(); + if (consumedCapacity.localSecondaryIndexes() != null) { for (Capacity lsiConsumedCapacity : - consumedCapacity.getLocalSecondaryIndexes().values()) { - consumedUnits += lsiConsumedCapacity.getCapacityUnits(); + consumedCapacity.localSecondaryIndexes().values()) { + consumedUnits += lsiConsumedCapacity.capacityUnits(); } } totalIOPSConsumed += consumedUnits; @@ -130,7 +132,7 @@ public void write(K key, V value) throws IOException { } int unprocessedItems = 0; - for (List requests : result.getUnprocessedItems().values()) { + for (List requests : response.unprocessedItems().values()) { unprocessedItems += requests.size(); } writesPerSecond += batchSize - unprocessedItems; diff --git a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculator.java b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculator.java index b30812eb..df415769 100644 --- a/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculator.java +++ b/emr-dynamodb-hadoop/src/main/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculator.java @@ -10,8 +10,6 @@ package org.apache.hadoop.dynamodb.write; -import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.google.common.base.Strings; import java.io.IOException; import org.apache.commons.logging.Log; @@ -22,6 +20,9 @@ import org.apache.hadoop.dynamodb.IopsCalculator; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputDescription; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class WriteIopsCalculator implements IopsCalculator { @@ -89,11 +90,11 @@ int calculateMaxMapTasks(int totalMapTasks) { private double getThroughput() { TableDescription tableDescription = dynamoDBClient.describeTable(tableName); - if (tableDescription.getBillingModeSummary() == null || tableDescription.getBillingModeSummary() - .getBillingMode().equalsIgnoreCase(DynamoDBConstants.BILLING_MODE_PROVISIONED)) { + if (tableDescription.billingModeSummary() == null + || tableDescription.billingModeSummary().billingMode() == BillingMode.PROVISIONED) { ProvisionedThroughputDescription provisionedThroughput = - tableDescription.getProvisionedThroughput(); - return provisionedThroughput.getWriteCapacityUnits(); + tableDescription.provisionedThroughput(); + return provisionedThroughput.writeCapacityUnits(); } return DynamoDBConstants.DEFAULT_CAPACITY_FOR_ON_DEMAND; } diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBClientTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBClientTest.java index dc77a15b..50a8f9b7 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBClientTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBClientTest.java @@ -15,18 +15,9 @@ import static org.apache.hadoop.dynamodb.DynamoDBConstants.DEFAULT_MAX_ITEM_SIZE; -import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult; -import com.amazonaws.services.dynamodbv2.model.WriteRequest; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSSessionCredentials; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.hamcrest.core.Is; @@ -38,6 +29,13 @@ import java.util.List; import java.util.Map; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.WriteRequest; public class DynamoDBClientTest { @@ -50,13 +48,11 @@ public class DynamoDBClientTest { public ExpectedException expectedException = ExpectedException.none(); Configuration conf = new Configuration(); - ClientConfiguration clientConf; DynamoDBClient client; @Before public void setup() { conf.clear(); - clientConf = new ClientConfiguration(); client = new DynamoDBClient(conf); } @@ -69,9 +65,9 @@ public void testDynamoDBCredentials() { conf.set(DynamoDBConstants.DYNAMODB_SECRET_KEY_CONF, DYNAMODB_SECRET_KEY); DynamoDBClient dynamoDBClient = new DynamoDBClient(); - AWSCredentialsProvider provider = dynamoDBClient.getAWSCredentialsProvider(conf); - Assert.assertEquals(DYNAMODB_ACCESS_KEY, provider.getCredentials().getAWSAccessKeyId()); - Assert.assertEquals(DYNAMODB_SECRET_KEY, provider.getCredentials().getAWSSecretKey()); + AwsCredentialsProvider provider = dynamoDBClient.getAwsCredentialsProvider(conf); + Assert.assertEquals(DYNAMODB_ACCESS_KEY, provider.resolveCredentials().accessKeyId()); + Assert.assertEquals(DYNAMODB_SECRET_KEY, provider.resolveCredentials().secretAccessKey()); } @Test @@ -83,9 +79,9 @@ public void testDefaultCredentials() { conf.set(DynamoDBConstants.DEFAULT_SECRET_KEY_CONF, DEFAULT_SECRET_KEY); DynamoDBClient dynamoDBClient = new DynamoDBClient(); - AWSCredentialsProvider provider = dynamoDBClient.getAWSCredentialsProvider(conf); - Assert.assertEquals(DEFAULT_ACCESS_KEY, provider.getCredentials().getAWSAccessKeyId()); - Assert.assertEquals(DEFAULT_SECRET_KEY, provider.getCredentials().getAWSSecretKey()); + AwsCredentialsProvider provider = dynamoDBClient.getAwsCredentialsProvider(conf); + Assert.assertEquals(DEFAULT_ACCESS_KEY, provider.resolveCredentials().accessKeyId()); + Assert.assertEquals(DEFAULT_SECRET_KEY, provider.resolveCredentials().secretAccessKey()); } @Test @@ -99,9 +95,9 @@ public void testCustomCredentialsProvider() { .getName()); DynamoDBClient dynamoDBClient = new DynamoDBClient(); - AWSCredentialsProvider provider = dynamoDBClient.getAWSCredentialsProvider(conf); - Assert.assertEquals(MY_ACCESS_KEY, provider.getCredentials().getAWSAccessKeyId()); - Assert.assertEquals(MY_SECRET_KEY, provider.getCredentials().getAWSSecretKey()); + AwsCredentialsProvider provider = dynamoDBClient.getAwsCredentialsProvider(conf); + Assert.assertEquals(MY_ACCESS_KEY, provider.resolveCredentials().accessKeyId()); + Assert.assertEquals(MY_SECRET_KEY, provider.resolveCredentials().secretAccessKey()); } @Test @@ -111,7 +107,7 @@ public void testCustomProviderNotFound() { ".NonExistentCredentialsProvider"); DynamoDBClient dynamoDBClient = new DynamoDBClient(); expectedException.expectCause(Is.isA(ClassNotFoundException.class)); - dynamoDBClient.getAWSCredentialsProvider(conf); + dynamoDBClient.getAwsCredentialsProvider(conf); } @Test @@ -120,7 +116,7 @@ public void testCustomProviderCannotCast() { conf.set(DynamoDBConstants.CUSTOM_CREDENTIALS_PROVIDER_CONF, Object.class.getName()); DynamoDBClient dynamoDBClient = new DynamoDBClient(); expectedException.expect(ClassCastException.class); - dynamoDBClient.getAWSCredentialsProvider(conf); + dynamoDBClient.getAwsCredentialsProvider(conf); } @Test @@ -134,87 +130,91 @@ public void testBasicSessionCredentials(){ conf.set(DynamoDBConstants.DYNAMODB_SESSION_TOKEN_CONF, DYNAMODB_SESSION_KEY); DynamoDBClient dynamoDBClient = new DynamoDBClient(); - AWSCredentialsProvider provider = dynamoDBClient.getAWSCredentialsProvider(conf); - AWSSessionCredentials sessionCredentials = (AWSSessionCredentials) provider.getCredentials(); - Assert.assertEquals(DYNAMODB_ACCESS_KEY, sessionCredentials.getAWSAccessKeyId()); - Assert.assertEquals(DYNAMODB_SECRET_KEY, sessionCredentials.getAWSSecretKey()); - Assert.assertEquals(DYNAMODB_SESSION_KEY, sessionCredentials.getSessionToken()); + AwsCredentialsProvider provider = dynamoDBClient.getAwsCredentialsProvider(conf); + AwsSessionCredentials sessionCredentials = (AwsSessionCredentials) provider.resolveCredentials(); + Assert.assertEquals(DYNAMODB_ACCESS_KEY, sessionCredentials.accessKeyId()); + Assert.assertEquals(DYNAMODB_SECRET_KEY, sessionCredentials.secretAccessKey()); + Assert.assertEquals(DYNAMODB_SESSION_KEY, sessionCredentials.sessionToken()); } @Test public void setsClientConfigurationProxyHostAndPortWhenBothAreSupplied() { setTestProxyHostAndPort(conf); - client.applyProxyConfiguration(clientConf, conf); - Assert.assertEquals(TEST_PROXY_HOST, clientConf.getProxyHost()); - Assert.assertEquals(TEST_PROXY_PORT, clientConf.getProxyPort()); + ProxyConfiguration proxyConfig = client.applyProxyConfiguration(conf); + Assert.assertEquals(TEST_PROXY_HOST, proxyConfig.host()); + Assert.assertEquals(TEST_PROXY_PORT, proxyConfig.port()); } @Test(expected = RuntimeException.class) public void throwsWhenProxyPortIsMissing() { setProxyHostAndPort(conf, "test.proxy.host", 0); - client.applyProxyConfiguration(clientConf, conf); + client.applyProxyConfiguration(conf); } @Test(expected = RuntimeException.class) public void throwsWhenProxyHostIsMissing() { setProxyHostAndPort(conf, null, 5555); - client.applyProxyConfiguration(clientConf, conf); + client.applyProxyConfiguration(conf); } @Test public void setsClientConfigurationProxyUsernameAndPasswordWhenBothAreSuppliedWithProxyHostAndPort() { setTestProxyHostAndPort(conf); - setProxyUsernameAndPassword(conf, "username", "password"); - client.applyProxyConfiguration(clientConf, conf); + setProxyUsernameAndPassword(conf, TEST_USERNAME, TEST_PASSWORD); + ProxyConfiguration proxyConfig = client.applyProxyConfiguration(conf); + Assert.assertEquals(TEST_PROXY_HOST, proxyConfig.host()); + Assert.assertEquals(TEST_PROXY_PORT, proxyConfig.port()); + Assert.assertEquals(TEST_USERNAME, proxyConfig.username()); + Assert.assertEquals(TEST_PASSWORD, proxyConfig.password()); } @Test(expected = RuntimeException.class) public void throwsWhenProxyUsernameIsMissing() { setTestProxyHostAndPort(conf); - setProxyUsernameAndPassword(conf, null, "password"); - client.applyProxyConfiguration(clientConf, conf); + setProxyUsernameAndPassword(conf, null, TEST_PASSWORD); + client.applyProxyConfiguration(conf); } @Test(expected = RuntimeException.class) public void throwsWhenProxyPasswordIsMissing() { setTestProxyHostAndPort(conf); - conf.set(DynamoDBConstants.PROXY_USERNAME, "username"); - client.applyProxyConfiguration(clientConf, conf); + conf.set(DynamoDBConstants.PROXY_USERNAME, TEST_USERNAME); + client.applyProxyConfiguration(conf); } @Test(expected = RuntimeException.class) public void throwsWhenGivenProxyUsernameAndPasswordWithoutProxyHostAndPortAreNotSupplied() { setProxyUsernameAndPassword(conf, TEST_USERNAME, TEST_PASSWORD); - client.applyProxyConfiguration(clientConf, conf); + client.applyProxyConfiguration(conf); } @Test(expected = RuntimeException.class) public void testPutBatchThrowsWhenItemIsTooLarge() throws Exception { Map item = ImmutableMap.of("", - new AttributeValue(Strings.repeat("a", (int) (DEFAULT_MAX_ITEM_SIZE + 1)))); + AttributeValue.fromS(Strings.repeat("a", (int) (DEFAULT_MAX_ITEM_SIZE + 1)))); client.putBatch("dummyTable", item, 1, null, false); } @Test public void testPutBatchDoesNotThrowWhenItemIsNotTooLarge() throws Exception { Map item = ImmutableMap.of("", - new AttributeValue(Strings.repeat("a", (int) DEFAULT_MAX_ITEM_SIZE))); + AttributeValue.fromS(Strings.repeat("a", (int) DEFAULT_MAX_ITEM_SIZE))); client.putBatch("dummyTable", item, 1, null, false); } @Test public void testPutBatchDeletionModeSuccessful() throws Exception { Map item = ImmutableMap.of("", - new AttributeValue(Strings.repeat("a", (int) DEFAULT_MAX_ITEM_SIZE))); + AttributeValue.fromS(Strings.repeat("a", (int) DEFAULT_MAX_ITEM_SIZE))); client.putBatch("dummyTable", item, 1, null, true); for (Map.Entry> entry: client.getWriteBatchMap().entrySet()) { for (WriteRequest req: entry.getValue()) { - Assert.assertNotNull(req.getDeleteRequest()); - Assert.assertNull(req.getPutRequest()); + Assert.assertNotNull(req.deleteRequest()); + Assert.assertNull(req.putRequest()); } } } @@ -222,8 +222,8 @@ public void testPutBatchDeletionModeSuccessful() throws Exception { @Test public void testPutBatchDeletionModeSuccessfulWithAdditionalKeysInItem() throws Exception { Map item = ImmutableMap.of( - "a", new AttributeValue().withS("a"), - "b", new AttributeValue().withS("b") + "a", AttributeValue.fromS("a"), + "b", AttributeValue.fromS("b") ); conf.set(DynamoDBConstants.DYNAMODB_TABLE_KEY_NAMES, "a"); @@ -232,10 +232,10 @@ public void testPutBatchDeletionModeSuccessfulWithAdditionalKeysInItem() throws for (Map.Entry> entry: client.getWriteBatchMap().entrySet()) { for (WriteRequest req: entry.getValue()) { - Assert.assertNotNull(req.getDeleteRequest()); - Assert.assertEquals(1, req.getDeleteRequest().getKey().size()); - Assert.assertTrue(req.getDeleteRequest().getKey().containsKey("a")); - Assert.assertNull(req.getPutRequest()); + Assert.assertNotNull(req.deleteRequest()); + Assert.assertEquals(1, req.deleteRequest().key().size()); + Assert.assertTrue(req.deleteRequest().key().containsKey("a")); + Assert.assertNull(req.putRequest()); } } } @@ -243,8 +243,8 @@ public void testPutBatchDeletionModeSuccessfulWithAdditionalKeysInItem() throws @Test public void testPutBatchDeletionFailsAsGivenItemDoesNotContainAnyKey() throws Exception { Map item = ImmutableMap.of( - "c", new AttributeValue().withS("a"), - "d", new AttributeValue().withS("b") + "c", AttributeValue.fromS("a"), + "d", AttributeValue.fromS("b") ); conf.set(DynamoDBConstants.DYNAMODB_TABLE_KEY_NAMES, "a,b"); @@ -275,7 +275,7 @@ private void setProxyUsernameAndPassword(Configuration conf, String username, St } } - private static class MyAWSCredentialsProvider implements AWSCredentialsProvider, Configurable { + private static class MyAWSCredentialsProvider implements AwsCredentialsProvider, Configurable { private Configuration conf; private String accessKey; private String secretKey; @@ -286,13 +286,8 @@ private void init() { } @Override - public AWSCredentials getCredentials() { - return new BasicAWSCredentials(accessKey, secretKey); - } - - @Override - public void refresh() { - + public AwsCredentials resolveCredentials() { + return AwsBasicCredentials.create(accessKey, secretKey); } @Override diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryerTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryerTest.java index 9c8a3320..d0c73dbb 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryerTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBFibonacciRetryerTest.java @@ -19,9 +19,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; - import org.joda.time.Duration; import org.junit.Test; import org.junit.runner.RunWith; @@ -30,6 +27,10 @@ import java.io.IOException; import java.util.concurrent.Callable; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.http.SdkHttpResponse; @RunWith(MockitoJUnitRunner.class) public class DynamoDBFibonacciRetryerTest { @@ -38,7 +39,7 @@ public class DynamoDBFibonacciRetryerTest { private Callable call; @Test - public void testSuceedCall() throws Exception { + public void testSucceedCall() throws Exception { DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); retryer.runWithRetry(call, null, null); verify(call).call(); @@ -46,9 +47,15 @@ public void testSuceedCall() throws Exception { @Test(expected = RuntimeException.class) public void testRetryThrottleException() throws Exception { - AmazonServiceException ase = new AmazonServiceException("Test"); - ase.setErrorCode("ProvisionedThroughputExceededException"); - ase.setStatusCode(400); + AwsServiceException ase = AwsServiceException.builder() + .message("Test") + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode("ProvisionedThroughputExceededException") + .sdkHttpResponse(SdkHttpResponse.builder() + .statusCode(400) + .build()) + .build()) + .build(); when(call.call()).thenThrow(ase); DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); @@ -62,9 +69,15 @@ public void testRetryThrottleException() throws Exception { @Test(expected = RuntimeException.class) public void testRetryableASEException() throws Exception { - AmazonServiceException ase = new AmazonServiceException("Test"); - ase.setErrorCode("ArbitRetryableException"); - ase.setStatusCode(500); + AwsServiceException ase = AwsServiceException.builder() + .message("Test") + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode("ArbitRetryableException") + .sdkHttpResponse(SdkHttpResponse.builder() + .statusCode(500) + .build()) + .build()) + .build(); when(call.call()).thenThrow(ase); DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); @@ -78,9 +91,15 @@ public void testRetryableASEException() throws Exception { @Test(expected = RuntimeException.class) public void testRetryableASEException2() throws Exception { - AmazonServiceException ase = new AmazonServiceException("Test"); - ase.setErrorCode("ArbitRetryableException"); - ase.setStatusCode(503); + AwsServiceException ase = AwsServiceException.builder() + .message("Test") + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode("ArbitRetryableException") + .sdkHttpResponse(SdkHttpResponse.builder() + .statusCode(503) + .build()) + .build()) + .build(); when(call.call()).thenThrow(ase); DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); @@ -94,9 +113,15 @@ public void testRetryableASEException2() throws Exception { @Test(expected = RuntimeException.class) public void testNonRetryableASEException() throws Exception { - AmazonServiceException ase = new AmazonServiceException("Test"); - ase.setErrorCode("ArbitNonRetryableException"); - ase.setStatusCode(400); + AwsServiceException ase = AwsServiceException.builder() + .message("Test") + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode("ArbitNonRetryableException") + .sdkHttpResponse(SdkHttpResponse.builder() + .statusCode(400) + .build()) + .build()) + .build(); when(call.call()).thenThrow(ase); DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); @@ -109,7 +134,7 @@ public void testNonRetryableASEException() throws Exception { @Test(expected = RuntimeException.class) public void testRetryACEException() throws Exception { - AmazonClientException ace = new AmazonClientException("Test"); + SdkException ace = SdkException.builder().message("Test").build(); when(call.call()).thenThrow(ace); DynamoDBFibonacciRetryer retryer = new DynamoDBFibonacciRetryer(Duration.standardSeconds(10)); diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBItemWritableTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBItemWritableTest.java index d9244af4..f4133596 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBItemWritableTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBItemWritableTest.java @@ -21,9 +21,6 @@ import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; - -import org.apache.commons.io.IOUtils; import org.junit.Before; import org.junit.Test; @@ -42,6 +39,8 @@ import java.util.List; import java.util.Map; import java.util.Random; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBItemWritableTest { @@ -61,12 +60,12 @@ public void testSerialization() throws IOException { item.write(out); outStream.close(); - String data = outStream.toString(); + final byte[] data = outStream.toByteArray(); item.setItem(null); assertNull(item.getItem()); - item.readFields(new DataInputStream(IOUtils.toInputStream(data))); + item.readFields(new DataInputStream(new ByteArrayInputStream(data))); checkReturnedItem(); } @@ -100,21 +99,21 @@ private void checkReturnedItem() { assertNotNull(item.getItem()); Map returnedData = item.getItem(); assertEquals(5, returnedData.size()); - assertEquals("test", returnedData.get("s").getS()); - assertEquals("1234", returnedData.get("n").getN()); - assertNull(returnedData.get("n").getS()); - assertEquals(0, returnedData.get("ss").getSS().size()); - assertEquals(3, returnedData.get("ns").getNS().size()); - assertEquals(2, returnedData.get("l").getL().size()); - - List ns = returnedData.get("ns").getNS(); + assertEquals("test", returnedData.get("s").s()); + assertEquals("1234", returnedData.get("n").n()); + assertNull(returnedData.get("n").s()); + assertEquals(0, returnedData.get("ss").ss().size()); + assertEquals(3, returnedData.get("ns").ns().size()); + assertEquals(2, returnedData.get("l").l().size()); + + List ns = returnedData.get("ns").ns(); assertEquals("1.0", ns.get(0)); assertEquals("1.10", ns.get(1)); assertEquals("2.0", ns.get(2)); - List l = returnedData.get("l").getL(); - assertEquals("1.0", l.get(0).getS()); - assertEquals("0", l.get(1).getS()); + List l = returnedData.get("l").l(); + assertEquals("1.0", l.get(0).s()); + assertEquals("0", l.get(1).s()); } @Test @@ -137,10 +136,11 @@ public void testBinarySerialization() { for (int i = 0; i < loop; i++) { Map map = new HashMap<>(); - map.put("hash", new AttributeValue().withB(byteBuffers.get(rnd.nextInt(totalByteArrays)))); - map.put("range", new AttributeValue().withB(byteBuffers.get(rnd.nextInt(totalByteArrays)))); - map.put("list", new AttributeValue().withBS(byteBuffers.get(rnd.nextInt(totalByteArrays)), - byteBuffers.get(rnd.nextInt(totalByteArrays)))); + map.put("hash", AttributeValue.fromB(SdkBytes.fromByteBuffer(byteBuffers.get(rnd.nextInt(totalByteArrays))))); + map.put("range", AttributeValue.fromB(SdkBytes.fromByteBuffer(byteBuffers.get(rnd.nextInt(totalByteArrays))))); + map.put("list", AttributeValue.fromBs(Arrays.asList( + SdkBytes.fromByteBuffer(byteBuffers.get(rnd.nextInt(totalByteArrays))), + SdkBytes.fromByteBuffer(byteBuffers.get(rnd.nextInt(totalByteArrays)))))); Map dynamoDBItem = gson.fromJson(gson.toJson(map, type), type); compare(map, dynamoDBItem); @@ -156,9 +156,9 @@ public void testMalformedJsonDeserialization() { item.readFieldsStream(malformedJson); Map attrValueMap = item.getItem(); - assertEquals("seattle", attrValueMap.get("attr1").getS()); + assertEquals("seattle", attrValueMap.get("attr1").s()); assertEquals(new HashSet<>(Arrays.asList("123", "456", "789")), new HashSet<>(attrValueMap - .get("attr2").getNS())); + .get("attr2").ns())); } private void compare(Map map, Map map2) { @@ -171,10 +171,10 @@ private void compare(Map map, Map l = new ArrayList<>(); - l.add(new AttributeValue("1.0")); - l.add(new AttributeValue("0")); + l.add(AttributeValue.fromS("1.0")); + l.add(AttributeValue.fromS("0")); Map sampleData = new HashMap<>(); - sampleData.put("s", new AttributeValue().withS("test")); - sampleData.put("n", new AttributeValue().withN("1234")); - sampleData.put("ss", new AttributeValue().withSS(ss)); - sampleData.put("ns", new AttributeValue().withNS(ns)); - sampleData.put("l", new AttributeValue().withL(l)); + sampleData.put("s", AttributeValue.fromS("test")); + sampleData.put("n", AttributeValue.fromN("1234")); + sampleData.put("ss", AttributeValue.fromSs(ss)); + sampleData.put("ns", AttributeValue.fromNs(ns)); + sampleData.put("l", AttributeValue.fromL(l)); item.setItem(sampleData); } diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBUtilTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBUtilTest.java index fe339934..65899f87 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBUtilTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/DynamoDBUtilTest.java @@ -23,13 +23,6 @@ import com.google.common.collect.Lists; -import com.amazonaws.AmazonClientException; -import com.amazonaws.regions.Region; -import com.amazonaws.regions.RegionUtils; -import com.amazonaws.regions.ServiceAbbreviations; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.util.EC2MetadataUtils; - import org.apache.hadoop.conf.Configuration; import org.junit.Before; import org.junit.Test; @@ -39,17 +32,21 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.io.UnsupportedEncodingException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.regions.internal.util.EC2MetadataUtils; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; @RunWith(PowerMockRunner.class) -@PrepareForTest({RegionUtils.class, EC2MetadataUtils.class}) +@PrepareForTest({EC2MetadataUtils.class}) public class DynamoDBUtilTest { final String TEST_ENDPOINT = "http://emr.test-dynamodb.endpoint"; final String TEST_REGION = "test-region"; + final String TEST_REGION_ID = "test-region-id"; Configuration conf = new Configuration(); Region region; @@ -58,25 +55,23 @@ public class DynamoDBUtilTest { private static final String TEST_STRING = "AfFLIHsycSvZoEhPPKHUrtwewDAlcD"; private static final String TEST_NUMBER = "3592.0001"; private static final List TEST_NUMBER_ARRAY = Lists.newArrayList("2.14748364", "1.23452487", "1.73904643"); - private static final List TEST_LIST = Lists.newArrayList(new AttributeValue(TEST_STRING), - new AttributeValue().withN(TEST_NUMBER)); + private static final List TEST_LIST = Lists.newArrayList(AttributeValue.fromS(TEST_STRING), + AttributeValue.fromN(TEST_NUMBER)); private static final List TEST_MAP_KEYS = Lists.newArrayList("mapString", "mapNumber"); @Before public void setUp() { - PowerMockito.spy(RegionUtils.class); PowerMockito.mockStatic(EC2MetadataUtils.class); - region = mock(Region.class); conf.clear(); } @Test public void testArrayItemSize() throws UnsupportedEncodingException { Map item = new HashMap<>(); - item.put(TEST_NAMES.get(0), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(1), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(2), new AttributeValue().withN(TEST_NUMBER)); - item.put(TEST_NAMES.get(3), new AttributeValue().withNS(TEST_NUMBER_ARRAY)); + item.put(TEST_NAMES.get(0), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(1), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(2), AttributeValue.fromN(TEST_NUMBER)); + item.put(TEST_NAMES.get(3), AttributeValue.fromNs(TEST_NUMBER_ARRAY)); List allStrings = Lists.newArrayList(TEST_STRING, TEST_STRING, TEST_NUMBER); allStrings.addAll(TEST_NAMES); @@ -89,10 +84,10 @@ public void testArrayItemSize() throws UnsupportedEncodingException { public void testListItemSize() throws UnsupportedEncodingException { Map item = new HashMap<>(); int expectedSize = 0; - item.put(TEST_NAMES.get(0), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(1), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(2), new AttributeValue().withN(TEST_NUMBER)); - item.put(TEST_NAMES.get(3), new AttributeValue().withL(TEST_LIST)); + item.put(TEST_NAMES.get(0), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(1), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(2), AttributeValue.fromN(TEST_NUMBER)); + item.put(TEST_NAMES.get(3), AttributeValue.fromL(TEST_LIST)); List allStrings = Lists.newArrayList(TEST_STRING, TEST_STRING, TEST_STRING, TEST_NUMBER, TEST_NUMBER); allStrings.addAll(TEST_NAMES); @@ -103,14 +98,14 @@ public void testListItemSize() throws UnsupportedEncodingException { @Test public void testMapItemSize() throws UnsupportedEncodingException { Map item = new HashMap<>(); - item.put(TEST_NAMES.get(0), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(1), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(2), new AttributeValue().withN(TEST_NUMBER)); + item.put(TEST_NAMES.get(0), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(1), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(2), AttributeValue.fromN(TEST_NUMBER)); Map attrMap = new HashMap<>(); - attrMap.put(TEST_MAP_KEYS.get(0), new AttributeValue(TEST_STRING)); - attrMap.put(TEST_MAP_KEYS.get(1), new AttributeValue().withN(TEST_NUMBER)); - item.put(TEST_NAMES.get(3), new AttributeValue().withM(attrMap)); + attrMap.put(TEST_MAP_KEYS.get(0), AttributeValue.fromS(TEST_STRING)); + attrMap.put(TEST_MAP_KEYS.get(1), AttributeValue.fromN(TEST_NUMBER)); + item.put(TEST_NAMES.get(3), AttributeValue.fromM(attrMap)); List allStrings = Lists.newArrayList(TEST_STRING, TEST_STRING, TEST_STRING, TEST_NUMBER, TEST_NUMBER); allStrings.addAll(TEST_NAMES); @@ -129,7 +124,7 @@ public void testEmptyItemSize() throws UnsupportedEncodingException { @Test public void testNullItemSize() throws UnsupportedEncodingException { Map item = new HashMap<>(); - item.put(null, new AttributeValue(TEST_STRING)); + item.put(null, AttributeValue.fromS(TEST_STRING)); item.put(TEST_NAMES.get(0), null); List allStrings = Lists.newArrayList(TEST_STRING, TEST_NAMES.get(0)); @@ -140,10 +135,10 @@ public void testNullItemSize() throws UnsupportedEncodingException { @Test public void testNumberItemSize() throws UnsupportedEncodingException { Map item = new HashMap<>(); - item.put(TEST_NAMES.get(0), new AttributeValue().withN(TEST_NUMBER)); - item.put(TEST_NAMES.get(1), new AttributeValue(TEST_STRING)); - item.put(TEST_NAMES.get(2), new AttributeValue().withN(TEST_NUMBER)); - item.put(TEST_NAMES.get(3), new AttributeValue().withNS(TEST_NUMBER_ARRAY)); + item.put(TEST_NAMES.get(0), AttributeValue.fromN(TEST_NUMBER)); + item.put(TEST_NAMES.get(1), AttributeValue.fromS(TEST_STRING)); + item.put(TEST_NAMES.get(2), AttributeValue.fromN(TEST_NUMBER)); + item.put(TEST_NAMES.get(3), AttributeValue.fromNs(TEST_NUMBER_ARRAY)); List allStrings = Lists.newArrayList(TEST_STRING, TEST_NUMBER, TEST_NUMBER); allStrings.addAll(TEST_NAMES); @@ -156,41 +151,41 @@ public void testNumberItemSize() throws UnsupportedEncodingException { public void getsEndpointFromConf() { conf.set(DynamoDBConstants.ENDPOINT, TEST_ENDPOINT); assertEquals(TEST_ENDPOINT, DynamoDBUtil.getDynamoDBEndpoint(conf, null)); - verify(region, never()).getServiceEndpoint(ServiceAbbreviations.Dynamodb); } @Test - public void getsEndpointFromRegion() { - when(RegionUtils.getRegion(TEST_REGION)).thenReturn(region); - when(region.getServiceEndpoint(ServiceAbbreviations.Dynamodb)).thenReturn(TEST_ENDPOINT); + public void getsRegionFromConf() { conf.set(DynamoDBConstants.REGION, TEST_REGION); - assertEquals(TEST_ENDPOINT, DynamoDBUtil.getDynamoDBEndpoint(conf, null)); - verify(region).getServiceEndpoint(ServiceAbbreviations.Dynamodb); + assertEquals(TEST_REGION, DynamoDBUtil.getDynamoDBRegion(conf, null)); } @Test - public void getsEndpointFromEc2InstanceRegion() { - when(EC2MetadataUtils.getEC2InstanceRegion()).thenReturn("ec2-instance-region"); - when(RegionUtils.getRegion("ec2-instance-region")).thenReturn(region); - when(region.getServiceEndpoint(ServiceAbbreviations.Dynamodb)).thenReturn(TEST_ENDPOINT); - assertEquals(TEST_ENDPOINT, DynamoDBUtil.getDynamoDBEndpoint(conf, null)); + public void getsRegionIdFromConf() { + conf.set(DynamoDBConstants.REGION_ID, TEST_REGION_ID); + assertEquals(TEST_REGION_ID, DynamoDBUtil.getDynamoDBRegion(conf, null)); + } + + @Test + public void getsRegionFromEc2Instance() { + final String EC2_INSTANCE_REGION = "ec2-instance-region"; + when(EC2MetadataUtils.getEC2InstanceRegion()).thenReturn(EC2_INSTANCE_REGION); + assertEquals(EC2_INSTANCE_REGION, DynamoDBUtil.getDynamoDBRegion(conf, null)); PowerMockito.verifyStatic(); EC2MetadataUtils.getEC2InstanceRegion(); - verify(region, never()).getServiceEndpoint(TEST_REGION); } @Test - public void getsEndpointFromDefaultAwsRegion() { - PowerMockito.mockStatic(RegionUtils.class); - when(EC2MetadataUtils.getEC2InstanceRegion()).thenThrow(new AmazonClientException("Unable to " + - "get region from EC2 instance data")); - when(RegionUtils.getRegion(DynamoDBConstants.DEFAULT_AWS_REGION)).thenReturn(region); - when(region.getServiceEndpoint(ServiceAbbreviations.Dynamodb)).thenReturn(TEST_ENDPOINT); - assertEquals(TEST_ENDPOINT, DynamoDBUtil.getDynamoDBEndpoint(conf, null)); + public void getsRegionFromDefaultAwsRegion() { + when(EC2MetadataUtils.getEC2InstanceRegion()) + .thenThrow(SdkException.builder() + .message("Unable to get region from EC2 instance data") + .build()); + + assertEquals(DynamoDBConstants.DEFAULT_AWS_REGION, DynamoDBUtil.getDynamoDBRegion(conf, null)); PowerMockito.verifyStatic(); - RegionUtils.getRegion(DynamoDBConstants.DEFAULT_AWS_REGION); } + @Test public void testGetBoundedBatchLimit() { assertEquals(1, getBoundedBatchLimit(conf, 0)); diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequestTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequestTest.java index 867481ac..cf64dd72 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequestTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/preader/ScanRecordReadRequestTest.java @@ -7,9 +7,6 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.when; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.ScanResult; - import org.apache.hadoop.dynamodb.DynamoDBClient; import org.apache.hadoop.dynamodb.DynamoDBFibonacciRetryer.RetryResult; import org.apache.hadoop.dynamodb.filter.DynamoDBQueryFilter; @@ -25,6 +22,9 @@ import java.util.HashMap; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.ConsumedCapacity; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; @RunWith(MockitoJUnitRunner.class) public final class ScanRecordReadRequestTest { @@ -36,8 +36,10 @@ public final class ScanRecordReadRequestTest { @Test public void fetchPageReturnsZeroConsumedCapacityWhenResultsConsumedCapacityIsNull() { - RetryResult stubbedResult = new RetryResult<>(new ScanResult().withConsumedCapacity(null) - .withItems(new HashMap()), 0); + RetryResult stubbedResult = new RetryResult<>(ScanResponse.builder() + .consumedCapacity((ConsumedCapacity) null) + .items(new HashMap()) + .build(), 0); stubScanTableWith(stubbedResult); when(context.getClient()).thenReturn(client); @@ -50,7 +52,7 @@ public void fetchPageReturnsZeroConsumedCapacityWhenResultsConsumedCapacityIsNul assertEquals(0.0, pageResults.consumedRcu, 0.0); } - private void stubScanTableWith(RetryResult scanResultRetryResult) { + private void stubScanTableWith(RetryResult scanResultRetryResult) { when(client.scanTable( anyString(), any(DynamoDBQueryFilter.class), diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/DynamoDBRecordReaderTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/DynamoDBRecordReaderTest.java index 78604fea..a5c306fd 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/DynamoDBRecordReaderTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/DynamoDBRecordReaderTest.java @@ -16,15 +16,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import com.amazonaws.services.dynamodbv2.model.AttributeDefinition; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.ConsumedCapacity; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.KeyType; -import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; -import com.amazonaws.services.dynamodbv2.model.ScanResult; -import com.amazonaws.services.dynamodbv2.model.TableDescription; - import org.apache.hadoop.dynamodb.DynamoDBClient; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.dynamodb.DynamoDBFibonacciRetryer.RetryResult; @@ -45,6 +36,14 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.ConsumedCapacity; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.KeyType; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputDescription; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class DynamoDBRecordReaderTest { @@ -70,23 +69,32 @@ public TableDescription describeTable(String tableName) { } @Override - public RetryResult scanTable(String tableName, DynamoDBQueryFilter + public RetryResult scanTable(String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { List> items = getItems(); if (i == 0) { Map lastEvaluatedKey = new HashMap<>(); - lastEvaluatedKey.put("test", new AttributeValue("test")); + lastEvaluatedKey.put("test", AttributeValue.fromS("test")); i++; - return new RetryResult<>(new ScanResult().withScannedCount(items.size()) - .withConsumedCapacity(new ConsumedCapacity().withCapacityUnits(1d)).withItems - (items).withLastEvaluatedKey(lastEvaluatedKey), 0); + return new RetryResult<>(ScanResponse.builder() + .scannedCount(items.size()) + .consumedCapacity(ConsumedCapacity.builder() + .capacityUnits(1d) + .build()) + .items(items) + .lastEvaluatedKey(lastEvaluatedKey) + .build(), 0); } else { - assertEquals("test", exclusiveStartKey.get("test").getS()); - return new RetryResult<>(new ScanResult().withScannedCount(items.size()) - .withConsumedCapacity(new ConsumedCapacity().withCapacityUnits(1d)).withItems - (items), 0); + assertEquals("test", exclusiveStartKey.get("test").s()); + return new RetryResult<>(ScanResponse.builder() + .scannedCount(items.size()) + .consumedCapacity(ConsumedCapacity.builder() + .capacityUnits(1d) + .build()) + .items(items) + .build(), 0); } } @@ -94,7 +102,7 @@ private List> getItems() { List> items = new ArrayList<>(); for (String key : HASH_KEYS) { Map item = new HashMap<>(); - item.put("hashKey", new AttributeValue(key)); + item.put("hashKey", AttributeValue.fromS(key)); items.add(item); } return items; @@ -133,7 +141,7 @@ public TableDescription describeTable(String tableName) { } @Override - public RetryResult scanTable(String tableName, DynamoDBQueryFilter + public RetryResult scanTable(String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { return new RetryResult<>(getHashNumberRangeKeyItems(HASH_KEYS, "S"), 0); @@ -175,7 +183,7 @@ public TableDescription describeTable(String tableName) { } @Override - public RetryResult scanTable(String tableName, DynamoDBQueryFilter + public RetryResult scanTable(String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { assertNull(exclusiveStartKey); @@ -217,7 +225,7 @@ public TableDescription describeTable(String tableName) { } @Override - public RetryResult scanTable(String tableName, DynamoDBQueryFilter + public RetryResult scanTable(String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { assertEquals(0, (int) segment); @@ -256,7 +264,7 @@ public TableDescription describeTable(String tableName) { } @Override - public RetryResult scanTable(String tableName, DynamoDBQueryFilter + public RetryResult scanTable(String tableName, DynamoDBQueryFilter dynamoDBQueryFilter, Integer segment, Integer totalSegments, Map exclusiveStartKey, long limit, Reporter reporter) { throw new RuntimeException("Unrecoverable Exception"); @@ -325,68 +333,91 @@ private String getRangeKeyValue(DynamoDBItemWritable value, String type) { private String getKeyValue(DynamoDBItemWritable value, String keyName, String type) { if (type.equals("S")) { - return value.getItem().get(keyName).getS(); + return value.getItem().get(keyName).s(); } else { - return value.getItem().get(keyName).getN(); + return value.getItem().get(keyName).n(); } } - private ScanResult getHashKeyItems(String[] hashKeys) { + private ScanResponse getHashKeyItems(String[] hashKeys) { return getHashKeyItems(hashKeys, "S"); } - private ScanResult getHashKeyItems(String[] hashKeys, String type) { + private ScanResponse getHashKeyItems(String[] hashKeys, String type) { List> items = new ArrayList<>(); for (String key : hashKeys) { Map item = new HashMap<>(); if (type.equals("S")) { - item.put("hashKey", new AttributeValue(key)); + item.put("hashKey", AttributeValue.fromS(key)); } else { - item.put("hashKey", new AttributeValue().withN(key)); + item.put("hashKey", AttributeValue.fromN(key)); } items.add(item); } - return new ScanResult().withScannedCount(items.size()).withItems(items).withConsumedCapacity - (new ConsumedCapacity().withCapacityUnits(1d)); + return ScanResponse.builder() + .scannedCount(items.size()) + .items(items) + .consumedCapacity(ConsumedCapacity.builder() + .capacityUnits(1d) + .build()) + .build(); } - private ScanResult getHashNumberRangeKeyItems(String[] hashKeys, String hashType) { + private ScanResponse getHashNumberRangeKeyItems(String[] hashKeys, String hashType) { List> items = new ArrayList<>(); for (String key : hashKeys) { for (Integer i = 0; i < NUM_RANGE_KEYS_PER_HASH_KEY; i++) { Map item = new HashMap<>(); if (hashType.equals("S")) { - item.put("hashKey", new AttributeValue(key)); + item.put("hashKey", AttributeValue.fromS(key)); } else { - item.put("hashKey", new AttributeValue().withN(key)); + item.put("hashKey", AttributeValue.fromN(key)); } - item.put("rangeKey", new AttributeValue().withN("0" + i.toString())); + item.put("rangeKey", AttributeValue.fromN("0" + i.toString())); items.add(item); } } - return new ScanResult().withScannedCount(items.size()).withItems(items).withConsumedCapacity - (new ConsumedCapacity().withCapacityUnits(1d)); + return ScanResponse.builder() + .scannedCount(items.size()) + .items(items) + .consumedCapacity(ConsumedCapacity.builder() + .capacityUnits(1d) + .build()) + .build(); } private TableDescription getTableDescription(String hashType, String rangeType) { List keySchema = new ArrayList<>(); List definitions = new ArrayList<>(); - keySchema.add(new KeySchemaElement().withAttributeName("hashKey").withKeyType(KeyType.HASH)); - definitions.add(new AttributeDefinition().withAttributeName("hashKey").withAttributeType - (hashType)); + keySchema.add(KeySchemaElement.builder() + .attributeName("hashKey") + .keyType(KeyType.HASH) + .build()); + definitions.add(AttributeDefinition.builder() + .attributeName("hashKey") + .attributeType(hashType) + .build()); if (rangeType != null) { - keySchema.add(new KeySchemaElement().withAttributeName("rangeKey").withKeyType(KeyType - .RANGE)); - definitions.add(new AttributeDefinition().withAttributeName("rangeKey").withAttributeType - (rangeType)); + keySchema.add(KeySchemaElement.builder() + .attributeName("rangeKey") + .keyType(KeyType.RANGE) + .build()); + definitions.add(AttributeDefinition.builder() + .attributeName("rangeKey") + .attributeType(rangeType) + .build()); } - TableDescription description = new TableDescription().withKeySchema(keySchema) - .withAttributeDefinitions(definitions).withProvisionedThroughput(new - ProvisionedThroughputDescription().withReadCapacityUnits(1000L) - .withWriteCapacityUnits(1000L)); + TableDescription description = TableDescription.builder() + .keySchema(keySchema) + .attributeDefinitions(definitions) + .provisionedThroughput(ProvisionedThroughputDescription.builder() + .readCapacityUnits(1000L) + .writeCapacityUnits(1000L) + .build()) + .build(); return description; } diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculatorTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculatorTest.java index 2864b396..0c6c035e 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculatorTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/read/ReadIopsCalculatorTest.java @@ -15,11 +15,6 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; - -import com.amazonaws.services.dynamodbv2.model.BillingModeSummary; -import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; -import com.amazonaws.services.dynamodbv2.model.TableDescription; - import org.apache.hadoop.dynamodb.DynamoDBClient; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.mapred.JobClient; @@ -29,6 +24,9 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.dynamodb.model.BillingModeSummary; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputDescription; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; @RunWith(MockitoJUnitRunner.class) public class ReadIopsCalculatorTest { @@ -47,11 +45,14 @@ public class ReadIopsCalculatorTest { @Before public void setup() { - when(dynamoDBClient.describeTable(TABLE_NAME)).thenReturn(new TableDescription() - .withBillingModeSummary( - new BillingModeSummary().withBillingMode(DynamoDBConstants.BILLING_MODE_PROVISIONED)) - .withProvisionedThroughput( - new ProvisionedThroughputDescription().withReadCapacityUnits(READ_CAPACITY_UNITS))); + when(dynamoDBClient.describeTable(TABLE_NAME)).thenReturn(TableDescription.builder() + .billingModeSummary(BillingModeSummary.builder() + .billingMode(DynamoDBConstants.BILLING_MODE_PROVISIONED) + .build()) + .provisionedThroughput(ProvisionedThroughputDescription.builder() + .readCapacityUnits(READ_CAPACITY_UNITS) + .build()) + .build()); JobConf jobConf = new JobConf(); jobConf.set(DynamoDBConstants.THROUGHPUT_READ_PERCENT, String.valueOf(THROUGHPUT_READ_PERCENT)); diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/DynamoDBTestUtils.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/DynamoDBTestUtils.java index 99080f9a..b7356349 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/DynamoDBTestUtils.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/DynamoDBTestUtils.java @@ -13,17 +13,15 @@ package org.apache.hadoop.dynamodb.test; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.dynamodb.type.DynamoDBTypeConstants; import com.google.common.collect.Lists; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; +import java.util.*; +import java.util.stream.Collectors; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import static org.junit.Assert.assertEquals; @@ -103,13 +101,15 @@ public static List getRandomByteBuffers() { public static Map getRandomItem() { Map item = new HashMap<>(); - item.put(DynamoDBTypeConstants.STRING, new AttributeValue().withS(getRandomString())); - item.put(DynamoDBTypeConstants.STRING_SET, new AttributeValue().withSS(getRandomStrings())); - item.put(DynamoDBTypeConstants.NUMBER, new AttributeValue().withN(getRandomNumber())); - item.put(DynamoDBTypeConstants.LIST, new AttributeValue().withL(new AttributeValue().withM(aRandomMap))); - item.put(DynamoDBTypeConstants.NUMBER_SET, new AttributeValue().withNS(getRandomNumbers())); - item.put(DynamoDBTypeConstants.BINARY, new AttributeValue().withB(getRandomByteBuffer())); - item.put(DynamoDBTypeConstants.BINARY_SET, new AttributeValue().withBS(getRandomByteBuffers())); + item.put(DynamoDBTypeConstants.STRING, AttributeValue.fromS(getRandomString())); + item.put(DynamoDBTypeConstants.STRING_SET, AttributeValue.fromSs(getRandomStrings())); + item.put(DynamoDBTypeConstants.NUMBER, AttributeValue.fromN(getRandomNumber())); + item.put(DynamoDBTypeConstants.LIST, AttributeValue.fromL(Arrays.asList(AttributeValue.fromM(aRandomMap)))); + item.put(DynamoDBTypeConstants.NUMBER_SET, AttributeValue.fromNs(getRandomNumbers())); + item.put(DynamoDBTypeConstants.BINARY, AttributeValue.fromB(SdkBytes.fromByteBuffer(getRandomByteBuffer()))); + item.put(DynamoDBTypeConstants.BINARY_SET, AttributeValue.fromBs(getRandomByteBuffers().stream() + .map(byteBuffer -> SdkBytes.fromByteBuffer(byteBuffer)) + .collect(Collectors.toList()))); return item; } diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/GsonTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/GsonTest.java index 33d5d8b7..9deef6d9 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/GsonTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/test/GsonTest.java @@ -17,8 +17,6 @@ import com.google.gson.Gson; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; - import org.apache.hadoop.dynamodb.DynamoDBItemWritable; import org.apache.hadoop.dynamodb.DynamoDBUtil; import org.junit.Test; @@ -28,6 +26,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class GsonTest { private static final int TASK_COUNT = 100; @@ -44,7 +43,7 @@ public void testSingleThread() { @Test public void testSpecialCase1() { Map item = DynamoDBTestUtils.getRandomItem(); - item.put("S", new AttributeValue().withS("This is a \n \0 \1 \2 line test")); + item.put("S", AttributeValue.fromS("This is a \n \0 \1 \2 line test")); Gson gson = DynamoDBUtil.getGson(); String json = gson.toJson(item, DynamoDBItemWritable.type); diff --git a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculatorTest.java b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculatorTest.java index b9af5baf..0e7899dc 100644 --- a/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculatorTest.java +++ b/emr-dynamodb-hadoop/src/test/java/org/apache/hadoop/dynamodb/write/WriteIopsCalculatorTest.java @@ -16,10 +16,6 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; -import com.amazonaws.services.dynamodbv2.model.BillingModeSummary; -import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; -import com.amazonaws.services.dynamodbv2.model.TableDescription; - import org.apache.hadoop.dynamodb.DynamoDBClient; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.mapred.JobClient; @@ -29,6 +25,10 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.BillingModeSummary; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputDescription; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; @RunWith(MockitoJUnitRunner.class) public class WriteIopsCalculatorTest { @@ -47,11 +47,16 @@ public class WriteIopsCalculatorTest { @Before public void setup() { - when(dynamoDBClient.describeTable(TABLE_NAME)).thenReturn(new TableDescription() - .withBillingModeSummary( - new BillingModeSummary().withBillingMode(DynamoDBConstants.BILLING_MODE_PROVISIONED)) - .withProvisionedThroughput( - new ProvisionedThroughputDescription().withWriteCapacityUnits(WRITE_CAPACITY_UNITS))); + when(dynamoDBClient.describeTable(TABLE_NAME)).thenReturn(TableDescription.builder() + .billingModeSummary( + BillingModeSummary.builder() + .billingMode(BillingMode.PROVISIONED) + .build()) + .provisionedThroughput( + ProvisionedThroughputDescription.builder() + .writeCapacityUnits(WRITE_CAPACITY_UNITS) + .build()) + .build()); JobConf jobConf = new JobConf(); jobConf.setNumMapTasks(TOTAL_MAP_TASKS); diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBExportSerDe.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBExportSerDe.java index 99348082..989ef6c0 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBExportSerDe.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBExportSerDe.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import com.google.common.base.Strings; import com.google.common.collect.Maps; import java.util.Arrays; @@ -27,6 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; /** * This class is used to read the DynamoDB backup format and allow querying individual columns from diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspector.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspector.java index efa8e841..946525b4 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspector.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspector.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.util.ArrayList; @@ -29,6 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBObjectInspector extends StructObjectInspector { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDe.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDe.java index d83365cc..c7a23ee6 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDe.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDe.java @@ -13,8 +13,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.BillingModeSummary; import com.google.common.collect.Maps; import java.io.IOException; import java.util.List; @@ -47,6 +45,9 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.BillingModeSummary; public class DynamoDBSerDe extends AbstractSerDe { @@ -193,8 +194,8 @@ private void verifyDynamoDBWriteThroughput(Configuration conf, Properties tbl) { log.info("Table Properties:" + tbl); DynamoDBClient client = new DynamoDBClient(conf, tbl.getProperty(DynamoDBConstants.REGION)); - long writesPerSecond = client.describeTable(dynamoDBTableName).getProvisionedThroughput() - .getWriteCapacityUnits(); + long writesPerSecond = client.describeTable(dynamoDBTableName).provisionedThroughput() + .writeCapacityUnits(); // skip verification when current resource manager is not Yarn if (!DynamoDBUtil.isYarnEnabled(conf)) { @@ -211,11 +212,10 @@ private void verifyDynamoDBWriteThroughput(Configuration conf, Properties tbl) { } BillingModeSummary billingModeSummary = - client.describeTable(dynamoDBTableName).getBillingModeSummary(); + client.describeTable(dynamoDBTableName).billingModeSummary(); if (maxMapTasks > writesPerSecond && (billingModeSummary == null - || billingModeSummary.getBillingMode().equals( - DynamoDBConstants.BILLING_MODE_PROVISIONED))) { + || billingModeSummary.billingMode() == BillingMode.PROVISIONED)) { String message = "WARNING: Configured write throughput of the dynamodb table " + dynamoDBTableName + " is less than the cluster map capacity." + " ClusterMapCapacity: " + maxMapTasks + " WriteThroughput: " + writesPerSecond + "\nWARNING: Writes to this " diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandler.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandler.java index 8cac276f..fb295546 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandler.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandler.java @@ -13,9 +13,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeDefinition; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.google.common.base.Strings; import java.util.HashMap; import java.util.List; @@ -54,6 +51,10 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputFormat; +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class DynamoDBStorageHandler implements HiveMetaHook, HiveStoragePredicateHandler, HiveStorageHandler { @@ -148,8 +149,8 @@ public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties.put(DynamoDBConstants.TABLE_NAME, tableName); jobProperties.put(DynamoDBConstants.DYNAMODB_TABLE_KEY_NAMES, - description.getKeySchema().stream() - .map(KeySchemaElement::getAttributeName) + description.keySchema().stream() + .map(KeySchemaElement::attributeName) .collect(Collectors.joining(DynamoDBConstants.DYNAMODB_TABLE_KEY_NAMES_SEPARATOR))); Map hiveToDynamoDBSchemaMapping = HiveDynamoDBUtil @@ -188,9 +189,8 @@ public void configureTableJobProperties(TableDesc tableDesc, Map .getProperty(DynamoDBConstants.THROUGHPUT_WRITE_PERCENT)); } - if (description.getBillingModeSummary() == null - || description.getBillingModeSummary().getBillingMode() - .equals(DynamoDBConstants.BILLING_MODE_PROVISIONED)) { + if (description.billingModeSummary() == null + || description.billingModeSummary().billingMode() == BillingMode.PROVISIONED) { useExplicitThroughputIfRequired(jobProperties, tableDesc); } else { // If not specified at the table level, set default value @@ -202,14 +202,14 @@ public void configureTableJobProperties(TableDesc tableDesc, Map DynamoDBConstants.DEFAULT_CAPACITY_FOR_ON_DEMAND.toString())); } - jobProperties.put(DynamoDBConstants.ITEM_COUNT, description.getItemCount().toString()); - jobProperties.put(DynamoDBConstants.TABLE_SIZE_BYTES, description.getTableSizeBytes() + jobProperties.put(DynamoDBConstants.ITEM_COUNT, description.itemCount().toString()); + jobProperties.put(DynamoDBConstants.TABLE_SIZE_BYTES, description.tableSizeBytes() .toString()); jobProperties.put(DynamoDBConstants.AVG_ITEM_SIZE, averageItemSize.toString()); log.info("Average item size: " + averageItemSize); - log.info("Item count: " + description.getItemCount()); - log.info("Table size: " + description.getTableSizeBytes()); + log.info("Item count: " + description.itemCount()); + log.info("Table size: " + description.tableSizeBytes()); log.info("Read throughput: " + jobProperties.get(DynamoDBConstants.READ_THROUGHPUT)); log.info("Write throughput: " + jobProperties.get(DynamoDBConstants.WRITE_THROUGHPUT)); @@ -361,10 +361,10 @@ void checkTableSchemaType(TableDescription tableDescription, Table table) throws } // validate key schema - for (AttributeDefinition definition : tableDescription.getAttributeDefinitions()) { - String attributeName = definition.getAttributeName(); + for (AttributeDefinition definition : tableDescription.attributeDefinitions()) { + String attributeName = definition.attributeName(); if (fieldName.equalsIgnoreCase(attributeName)) { - String attributeType = definition.getAttributeType(); + String attributeType = definition.attributeTypeAsString(); if (HiveDynamoDBTypeFactory.isHiveDynamoDBItemMapType(ddType) || (!ddType.getDynamoDBType().equals(attributeType))) { throw new MetaException("The key element " + fieldName + " does not match type. " @@ -381,10 +381,10 @@ private DynamoDBClient createDynamoDBClient(Table table) { } private void checkTableStatus(TableDescription tableDescription) throws MetaException { - String status = tableDescription.getTableStatus(); + String status = tableDescription.tableStatusAsString(); if ("CREATING".equals(status) || "DELETING".equals(status)) { - throw new MetaException("Table " + tableDescription.getTableName() + " is in state " + throw new MetaException("Table " + tableDescription.tableName() + " is in state " + status); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/AbstractDynamoDBFilter.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/AbstractDynamoDBFilter.java index 6734ef82..98a7b341 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/AbstractDynamoDBFilter.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/AbstractDynamoDBFilter.java @@ -13,9 +13,9 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.Condition; import org.apache.hadoop.dynamodb.filter.DynamoDBFilter; import org.apache.hadoop.dynamodb.filter.DynamoDBFilterOperator; +import software.amazon.awssdk.services.dynamodb.model.Condition; public abstract class AbstractDynamoDBFilter implements DynamoDBFilter { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBBinaryFilter.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBBinaryFilter.java index 8a30c6c8..3e1a6d79 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBBinaryFilter.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBBinaryFilter.java @@ -13,12 +13,12 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.Condition; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.dynamodb.filter.DynamoDBFilterOperator; import org.apache.hadoop.hive.dynamodb.type.HiveDynamoDBTypeFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.Condition; public class DynamoDBBinaryFilter extends AbstractDynamoDBFilter { @@ -33,12 +33,14 @@ public DynamoDBBinaryFilter(String columnName, DynamoDBFilterOperator operator, @Override public Condition createCondition() { - Condition condition = new Condition(); - condition.setComparisonOperator(operator.getDynamoDBName()); List attributeValueList = new ArrayList(); attributeValueList.add(HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(columnType) .getAttributeValue(columnValue)); - condition.setAttributeValueList(attributeValueList); + + Condition condition = Condition.builder() + .comparisonOperator(operator.getDynamoDBName()) + .attributeValueList(attributeValueList) + .build(); return condition; } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdown.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdown.java index 696d9a9a..67c44249 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdown.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdown.java @@ -13,11 +13,6 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.GlobalSecondaryIndexDescription; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.LocalSecondaryIndexDescription; -import com.amazonaws.services.dynamodbv2.model.Projection; -import com.amazonaws.services.dynamodbv2.model.ProjectionType; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -38,6 +33,12 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.serde.serdeConstants; +import software.amazon.awssdk.services.dynamodb.model.GlobalSecondaryIndexDescription; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.KeyType; +import software.amazon.awssdk.services.dynamodb.model.LocalSecondaryIndexDescription; +import software.amazon.awssdk.services.dynamodb.model.Projection; +import software.amazon.awssdk.services.dynamodb.model.ProjectionType; public class DynamoDBFilterPushdown { @@ -256,8 +257,8 @@ private DynamoDBQueryFilter getDynamoDBQueryFilter(List schema indexUseForQuery = getIndexUseForQuery( schema, globalSecondaryIndexes.stream() - .map(index -> new DynamoDBIndexInfo(index.getIndexName(), - index.getKeySchema(), index.getProjection())) + .map(index -> new DynamoDBIndexInfo(index.indexName(), + index.keySchema(), index.projection())) .collect(Collectors.toList()), hiveDynamoDBMapping, filterMap, @@ -275,8 +276,8 @@ private DynamoDBQueryFilter getDynamoDBQueryFilter(List schema indexUseForQuery = getIndexUseForQuery( schema, localSecondaryIndexes.stream() - .map(index -> new DynamoDBIndexInfo(index.getIndexName(), - index.getKeySchema(), index.getProjection())) + .map(index -> new DynamoDBIndexInfo(index.indexName(), + index.keySchema(), index.projection())) .collect(Collectors.toList()), hiveDynamoDBMapping, filterMap, @@ -329,19 +330,19 @@ && indexIncludesAllMappedAttributes(tableSchema, index, hiveDynamoDBMapping)) { private boolean indexIncludesAllMappedAttributes(List tableSchema, DynamoDBIndexInfo index, Map hiveDynamoDBMapping) { Projection indexProjection = index.getIndexProjection(); - if (ProjectionType.ALL.toString().equals(indexProjection.getProjectionType())) { + if (ProjectionType.ALL == indexProjection.projectionType()) { return true; } Set projectionAttributes = new HashSet<>(); for (KeySchemaElement keySchemaElement: tableSchema) { - projectionAttributes.add(keySchemaElement.getAttributeName()); + projectionAttributes.add(keySchemaElement.attributeName()); } for (KeySchemaElement keySchemaElement: index.getIndexSchema()) { - projectionAttributes.add(keySchemaElement.getAttributeName()); + projectionAttributes.add(keySchemaElement.attributeName()); } - if (ProjectionType.INCLUDE.toString().equals(indexProjection.getProjectionType())) { - projectionAttributes.addAll(indexProjection.getNonKeyAttributes()); + if (ProjectionType.INCLUDE == indexProjection.projectionType()) { + projectionAttributes.addAll(indexProjection.nonKeyAttributes()); } log.info("Checking if all mapped attributes " + hiveDynamoDBMapping.values() @@ -363,8 +364,8 @@ private List getDynamoDBFiltersFromSchema(List boolean hashKeyFilterExists = false; if (schema.size() > 0 - && DYNAMODB_KEY_TYPE_HASH.equals(schema.get(HASH_KEY_INDEX).getKeyType())) { - String hashKeyName = schema.get(HASH_KEY_INDEX).getAttributeName(); + && KeyType.HASH == schema.get(HASH_KEY_INDEX).keyType()) { + String hashKeyName = schema.get(HASH_KEY_INDEX).attributeName(); if (filterMap.containsKey(hashKeyName)) { DynamoDBFilter hashKeyFilter = filterMap.get(hashKeyName); if (DynamoDBFilterOperator.EQ.equals(hashKeyFilter.getOperator())) { @@ -375,7 +376,7 @@ private List getDynamoDBFiltersFromSchema(List } if (hashKeyFilterExists && schema.size() > 1) { - String rangeKeyName = schema.get(RANGE_KEY_INDEX).getAttributeName(); + String rangeKeyName = schema.get(RANGE_KEY_INDEX).attributeName(); if (filterMap.containsKey(rangeKeyName)) { DynamoDBFilter rangeKeyFilter = filterMap.get(rangeKeyName); if (eligibleOperatorsForRange.contains(rangeKeyFilter.getOperator())) { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBNAryFilter.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBNAryFilter.java index 4bc4d318..01046268 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBNAryFilter.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBNAryFilter.java @@ -13,12 +13,12 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.Condition; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.dynamodb.filter.DynamoDBFilterOperator; import org.apache.hadoop.hive.dynamodb.type.HiveDynamoDBTypeFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.Condition; public class DynamoDBNAryFilter extends AbstractDynamoDBFilter { @@ -32,16 +32,16 @@ public DynamoDBNAryFilter(String columnName, DynamoDBFilterOperator operator, St @Override public Condition createCondition() { - Condition condition = new Condition(); - condition.setComparisonOperator(operator.getDynamoDBName()); - List attributeValueList = new ArrayList<>(); for (String value : columnValues) { attributeValueList.add(HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(columnType) .getAttributeValue(value)); } - condition.setAttributeValueList(attributeValueList); + Condition condition = Condition.builder() + .comparisonOperator(operator.getDynamoDBName()) + .attributeValueList(attributeValueList) + .build(); return condition; } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBUnaryFilter.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBUnaryFilter.java index ed95a8ad..263667e3 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBUnaryFilter.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBUnaryFilter.java @@ -13,8 +13,8 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.Condition; import org.apache.hadoop.dynamodb.filter.DynamoDBFilterOperator; +import software.amazon.awssdk.services.dynamodb.model.Condition; public class DynamoDBUnaryFilter extends AbstractDynamoDBFilter { @@ -25,8 +25,9 @@ public DynamoDBUnaryFilter(String columnName, DynamoDBFilterOperator operator, S @Override public Condition createCondition() { - Condition condition = new Condition(); - condition.setComparisonOperator(operator.getDynamoDBName()); + Condition condition = Condition.builder() + .comparisonOperator(operator.getDynamoDBName()) + .build(); return condition; } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/read/HiveDynamoDBInputFormat.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/read/HiveDynamoDBInputFormat.java index cebb3c7d..2a47c196 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/read/HiveDynamoDBInputFormat.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/read/HiveDynamoDBInputFormat.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hive.dynamodb.read; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -41,6 +40,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class HiveDynamoDBInputFormat extends DynamoDBInputFormat { @@ -152,9 +152,9 @@ private DynamoDBQueryFilter getQueryFilter(JobConf conf, Map TableDescription tableDescription = client.describeTable(conf.get(DynamoDBConstants.TABLE_NAME)); DynamoDBQueryFilter queryFilter = pushdown.predicateToDynamoDBFilter( - tableDescription.getKeySchema(), - tableDescription.getLocalSecondaryIndexes(), - tableDescription.getGlobalSecondaryIndexes(), + tableDescription.keySchema(), + tableDescription.localSecondaryIndexes(), + tableDescription.globalSecondaryIndexes(), hiveDynamoDBMapping, hiveTypeMapping, filterExpr); return queryFilter; } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinarySetType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinarySetType.java index e029b2ea..d8fa7819 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinarySetType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinarySetType.java @@ -13,27 +13,31 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import org.apache.hadoop.dynamodb.type.DynamoDBBinarySetType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBBinarySetType extends DynamoDBBinarySetType implements HiveDynamoDBType { @Override public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspector, - boolean nullSerialization) { + boolean nullSerialization) { List values = DynamoDBDataParser.getByteBuffers(data, objectInspector, getDynamoDBType()); return (values == null || values.isEmpty()) ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withBS(values); + : AttributeValue.fromBs(values.stream() + .map(byteBuffer -> SdkBytes.fromByteBuffer(byteBuffer)) + .collect(Collectors.toList())); } @Override @@ -48,15 +52,15 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - List byteBuffers = data.getBS(); + List sdkBytesList = data.bs(); - if (byteBuffers == null || byteBuffers.isEmpty()) { + if (sdkBytesList == null || sdkBytesList.isEmpty()) { return null; } - List byteArrays = new ArrayList<>(byteBuffers.size()); - for (ByteBuffer byteBuffer : byteBuffers) { - byteArrays.add(Arrays.copyOf(byteBuffer.array(), byteBuffer.array().length)); + List byteArrays = new ArrayList<>(sdkBytesList.size()); + for (SdkBytes sdkBytes : sdkBytesList) { + byteArrays.add(Arrays.copyOf(sdkBytes.asByteArray(), sdkBytes.asByteArray().length)); } return byteArrays; diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinaryType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinaryType.java index b3724bb9..467a424b 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinaryType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBinaryType.java @@ -13,13 +13,14 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.nio.ByteBuffer; import org.apache.hadoop.dynamodb.type.DynamoDBBinaryType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBBinaryType extends DynamoDBBinaryType implements HiveDynamoDBType { @@ -29,7 +30,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect ByteBuffer value = DynamoDBDataParser.getByteBuffer(data, objectInspector); return value == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withB(value); + : AttributeValue.fromB(SdkBytes.fromByteBuffer(value)); } @Override @@ -44,6 +45,6 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getB() == null ? null : data.getB().array(); + return data.b() == null ? null : data.b().asByteArray(); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBooleanType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBooleanType.java index 8b7ad2b1..a9ce9e79 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBooleanType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBBooleanType.java @@ -1,11 +1,11 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.type.DynamoDBBooleanType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBBooleanType extends DynamoDBBooleanType implements HiveDynamoDBType { @@ -15,7 +15,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect Boolean value = DynamoDBDataParser.getBoolean(data, objectInspector); return value == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withBOOL(value); + : AttributeValue.fromBool(value); } @Override @@ -30,6 +30,6 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getBOOL(); + return data.bool(); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBItemType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBItemType.java index 56098b67..a7e41a14 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBItemType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBItemType.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import java.lang.reflect.Type; @@ -30,6 +29,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBItemType implements DynamoDBItemType, HiveDynamoDBType { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBListType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBListType.java index 12d54fe1..beb6c1bb 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBListType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBListType.java @@ -11,13 +11,13 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.List; import org.apache.hadoop.dynamodb.type.DynamoDBListType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBListType extends DynamoDBListType implements HiveDynamoDBType { @@ -28,7 +28,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect DynamoDBDataParser.getListAttribute(data, objectInspector, nullSerialization); return values == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withL(values); + : AttributeValue.fromL(values); } @Override @@ -54,8 +54,8 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getL() == null ? null - : DynamoDBDataParser.getListObject(data.getL(), objectInspector); + return data.hasL() ? DynamoDBDataParser.getListObject(data.l(), objectInspector) + : null; } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBMapType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBMapType.java index a41987b5..191ff347 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBMapType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBMapType.java @@ -11,7 +11,6 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.Map; import org.apache.hadoop.dynamodb.type.DynamoDBMapType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; @@ -20,6 +19,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBMapType extends DynamoDBMapType implements HiveDynamoDBType { @@ -30,7 +30,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect DynamoDBDataParser.getMapAttribute(data, objectInspector, nullSerialization); return values == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withM(values); + : AttributeValue.fromM(values); } @Override @@ -70,10 +70,11 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - Map dataMap = data.getM(); - if (dataMap == null) { + if (!data.hasM()) { return null; } + + Map dataMap = data.m(); switch (objectInspector.getCategory()) { case MAP: return DynamoDBDataParser.getMapObject(dataMap, objectInspector); diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNullType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNullType.java index 2023317f..05fb9ecf 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNullType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNullType.java @@ -1,11 +1,11 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.type.DynamoDBNullType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBNullType extends DynamoDBNullType implements HiveDynamoDBType { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberSetType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberSetType.java index f44c76dc..4c21d289 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberSetType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberSetType.java @@ -13,13 +13,13 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.List; import org.apache.hadoop.dynamodb.type.DynamoDBNumberSetType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBNumberSetType extends DynamoDBNumberSetType implements HiveDynamoDBType { @@ -30,7 +30,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect getDynamoDBType()); return (values == null || values.isEmpty()) ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withNS(values); + : AttributeValue.fromNs(values); } @Override @@ -48,8 +48,8 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getNS() == null ? null - : DynamoDBDataParser.getNumberObjectList(data.getNS(), objectInspector); + return data.hasNs() ? DynamoDBDataParser.getNumberObjectList(data.ns(), objectInspector) + : null; } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberType.java index 4fea59b6..30d6ae0d 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBNumberType.java @@ -13,12 +13,12 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.type.DynamoDBNumberType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBNumberType extends DynamoDBNumberType implements HiveDynamoDBType { @@ -28,7 +28,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect String value = DynamoDBDataParser.getNumber(data, objectInspector); return value == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withN(value); + : AttributeValue.fromN(value); } @Override @@ -45,8 +45,8 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getN() == null ? null - : DynamoDBDataParser.getNumberObject(data.getN(), objectInspector); + return data.n() == null ? null + : DynamoDBDataParser.getNumberObject(data.n(), objectInspector); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringSetType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringSetType.java index 1d069918..22b8448b 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringSetType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringSetType.java @@ -13,13 +13,13 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.util.List; import org.apache.hadoop.dynamodb.type.DynamoDBStringSetType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBStringSetType extends DynamoDBStringSetType implements HiveDynamoDBType { @@ -30,7 +30,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect DynamoDBDataParser.getSetAttribute(data, objectInspector, getDynamoDBType()); return (values == null || values.isEmpty()) ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue().withSS(values); + : AttributeValue.fromSs(values); } @Override @@ -45,7 +45,7 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getSS(); + return data.ss(); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringType.java index b7279e84..503e9eaf 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBStringType.java @@ -13,12 +13,12 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.type.DynamoDBStringType; import org.apache.hadoop.hive.dynamodb.util.DynamoDBDataParser; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class HiveDynamoDBStringType extends DynamoDBStringType implements HiveDynamoDBType { @Override @@ -27,7 +27,7 @@ public AttributeValue getDynamoDBData(Object data, ObjectInspector objectInspect String value = DynamoDBDataParser.getString(data, objectInspector); return value == null ? DynamoDBDataParser.getNullAttribute(nullSerialization) - : new AttributeValue(value); + : AttributeValue.fromS(value); } @Override @@ -42,7 +42,7 @@ public boolean supportsHiveType(TypeInfo typeInfo) { @Override public Object getHiveData(AttributeValue data, ObjectInspector objectInspector) { - return data.getS(); + return data.s(); } } diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBType.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBType.java index a83afd0f..e9840668 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBType.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBType.java @@ -13,10 +13,10 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.type.DynamoDBType; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public interface HiveDynamoDBType extends DynamoDBType { diff --git a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/util/DynamoDBDataParser.java b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/util/DynamoDBDataParser.java index da644c6e..63ee8dae 100644 --- a/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/util/DynamoDBDataParser.java +++ b/emr-dynamodb-hive/src/main/java/org/apache/hadoop/hive/dynamodb/util/DynamoDBDataParser.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hive.dynamodb.util; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -36,6 +35,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; import org.apache.hadoop.io.BytesWritable; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; public class DynamoDBDataParser { private static final Log log = LogFactory.getLog(DynamoDBDataParser.class); diff --git a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspectorTest.java b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspectorTest.java index b33b3a13..7cd5f1bf 100644 --- a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspectorTest.java +++ b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBObjectInspectorTest.java @@ -11,7 +11,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.DynamoDBItemWritable; import org.apache.hadoop.dynamodb.test.DynamoDBTestUtils; import org.apache.hadoop.dynamodb.type.DynamoDBTypeConstants; @@ -26,6 +25,7 @@ import java.util.List; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import static org.junit.Assert.assertEquals; @@ -64,10 +64,10 @@ public void testPrimitives() { expectedRowData.add(Boolean.valueOf(data.get(3))); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); - itemMap.put(attributeNames.get(1), new AttributeValue().withN(data.get(1))); - itemMap.put(attributeNames.get(2), new AttributeValue().withN(data.get(2))); - itemMap.put(attributeNames.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); + itemMap.put(attributeNames.get(1), AttributeValue.fromN(data.get(1))); + itemMap.put(attributeNames.get(2), AttributeValue.fromN(data.get(2))); + itemMap.put(attributeNames.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); List actualRowData = getDeserializedRow(attributeNames, colTypeInfos, itemMap); assertEquals(expectedRowData, actualRowData); @@ -85,8 +85,8 @@ public void testNull() { expectedRowData.addAll(data); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); - itemMap.put(attributeNames.get(1), new AttributeValue().withNULL(true)); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); + itemMap.put(attributeNames.get(1), AttributeValue.fromNul(true)); List actualRowData = getDeserializedRow(attributeNames, colTypeInfos, itemMap); assertEquals(expectedRowData, actualRowData); @@ -101,13 +101,13 @@ public void testArray() { List items = Lists.newArrayList("milk", "bread", "eggs", "milk"); List itemsAV = Lists.newArrayList(); for (String item : items) { - itemsAV.add(new AttributeValue(item)); + itemsAV.add(AttributeValue.fromS(item)); } List expectedRowData = Lists.newArrayList(list, items); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(list)); - itemMap.put(attributeNames.get(1), new AttributeValue().withL(itemsAV)); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(list)); + itemMap.put(attributeNames.get(1), AttributeValue.fromL(itemsAV)); List actualRowData = getDeserializedRow(attributeNames, colTypeInfos, itemMap); assertEquals(expectedRowData, actualRowData); @@ -117,7 +117,7 @@ public void testArray() { typeMapping.put(attributeNames.get(1), HiveDynamoDBTypeFactory.getTypeObjectFromDynamoDBType(DynamoDBTypeConstants.STRING_SET)); items = items.subList(0, 3); - itemMap.put(attributeNames.get(1), new AttributeValue().withSS(items)); + itemMap.put(attributeNames.get(1), AttributeValue.fromSs(items)); expectedRowData.set(1, items); actualRowData = getDeserializedRow(attributeNames, colTypeInfos, typeMapping, itemMap); @@ -140,18 +140,18 @@ public void testMap() { String person = people.get(i); long id = (long) i; - peopleAV.add(new AttributeValue(person)); + peopleAV.add(AttributeValue.fromS(person)); ids.put(person, id); - idsAV.put(person, new AttributeValue().withN(Long.toString(id))); + idsAV.put(person, AttributeValue.fromN(Long.toString(id))); lists.put(person, people.subList(0, i + 1)); - listsAV.put(person, new AttributeValue().withL(Lists.newArrayList(peopleAV))); + listsAV.put(person, AttributeValue.fromL(Lists.newArrayList(peopleAV))); } List expectedRowData = Lists.newArrayList(map, ids, lists); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(map)); - itemMap.put(attributeNames.get(1), new AttributeValue().withM(idsAV)); - itemMap.put(attributeNames.get(2), new AttributeValue().withM(listsAV)); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(map)); + itemMap.put(attributeNames.get(1), AttributeValue.fromM(idsAV)); + itemMap.put(attributeNames.get(2), AttributeValue.fromM(listsAV)); List actualRowData = getDeserializedRow(attributeNames, colTypeInfos, itemMap); assertEquals(expectedRowData, actualRowData); @@ -166,9 +166,9 @@ public void testMap() { Map namesAV = Maps.newHashMap(); for (String person : people) { names.put(person, person); - namesAV.put(person, new AttributeValue(person)); + namesAV.put(person, AttributeValue.fromS(person)); } - itemMap.put(attributeNames.get(3), new AttributeValue().withM(namesAV)); + itemMap.put(attributeNames.get(3), AttributeValue.fromM(namesAV)); expectedRowData.add(names); actualRowData = getDeserializedRow(attributeNames, colTypeInfos, altTypeMapping, itemMap); @@ -194,14 +194,14 @@ public void testStruct() { expectedRowData.add(structData); Map dataAV = Maps.newHashMap(); - dataAV.put(PRIMITIVE_FIELDS.get(0), new AttributeValue(data.get(0))); - dataAV.put(PRIMITIVE_FIELDS.get(1), new AttributeValue().withN(data.get(1))); - dataAV.put(PRIMITIVE_FIELDS.get(2), new AttributeValue().withN(data.get(2))); - dataAV.put(PRIMITIVE_FIELDS.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + dataAV.put(PRIMITIVE_FIELDS.get(0), AttributeValue.fromS(data.get(0))); + dataAV.put(PRIMITIVE_FIELDS.get(1), AttributeValue.fromN(data.get(1))); + dataAV.put(PRIMITIVE_FIELDS.get(2), AttributeValue.fromN(data.get(2))); + dataAV.put(PRIMITIVE_FIELDS.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(struct)); - itemMap.put(attributeNames.get(1), new AttributeValue().withM(dataAV)); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(struct)); + itemMap.put(attributeNames.get(1), AttributeValue.fromM(dataAV)); List actualRowData = getDeserializedRow(attributeNames, colTypeInfos, itemMap); @@ -234,10 +234,10 @@ public void testItem() { expectedRowData.add(colItemMap); Map itemMap = Maps.newHashMap(); - itemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); - itemMap.put(attributeNames.get(1), new AttributeValue().withN(data.get(1))); - itemMap.put(attributeNames.get(2), new AttributeValue().withN(data.get(2))); - itemMap.put(attributeNames.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + itemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); + itemMap.put(attributeNames.get(1), AttributeValue.fromN(data.get(1))); + itemMap.put(attributeNames.get(2), AttributeValue.fromN(data.get(2))); + itemMap.put(attributeNames.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); List actualRowData = getDeserializedRow(colNames, colTypeInfos, itemMap); diff --git a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDeTest.java b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDeTest.java index bab6b2e8..ca58f8b1 100644 --- a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDeTest.java +++ b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBSerDeTest.java @@ -11,7 +11,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.dynamodb.DynamoDBItemWritable; @@ -33,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import static org.junit.Assert.assertEquals; @@ -70,10 +70,10 @@ public void testPrimitives() throws SerDeException { List data = PRIMITIVE_STRING_DATA; Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withN(data.get(1))); - expectedItemMap.put(attributeNames.get(2), new AttributeValue().withN(data.get(2))); - expectedItemMap.put(attributeNames.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromN(data.get(1))); + expectedItemMap.put(attributeNames.get(2), AttributeValue.fromN(data.get(2))); + expectedItemMap.put(attributeNames.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); List rowData = Lists.newArrayList(); rowData.add(data.get(0)); @@ -94,7 +94,7 @@ public void testNull() throws SerDeException { data.set(1, null); Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); List rowData = Lists.newArrayList(); rowData.addAll(data); @@ -104,7 +104,7 @@ public void testNull() throws SerDeException { assertEquals(expectedItemMap, actualItemMap); // with null serialization - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withNULL(true)); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromNul(true)); actualItemMap = getSerializedItem(attributeNames, colOIs, rowData, true); assertEquals(expectedItemMap, actualItemMap); } @@ -118,12 +118,12 @@ public void testArray() throws SerDeException { List items = Lists.newArrayList("milk", "bread", "eggs", "milk"); List itemsAV = Lists.newArrayList(); for (String item : items) { - itemsAV.add(new AttributeValue(item)); + itemsAV.add(AttributeValue.fromS(item)); } Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(list)); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withL(itemsAV)); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(list)); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromL(itemsAV)); List rowData = Lists.newArrayList(); rowData.add(list); @@ -137,7 +137,7 @@ public void testArray() throws SerDeException { typeMapping.put(attributeNames.get(1), DynamoDBTypeConstants.STRING_SET); items = items.subList(0, 3); rowData.set(1, items); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withSS(items)); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromSs(items)); actualItemMap = getSerializedItem(attributeNames, colOIs, typeMapping, rowData); assertEquals(expectedItemMap, actualItemMap); @@ -160,17 +160,17 @@ public void testMap() throws SerDeException { String person = people.get(i); long id = (long) i; - peopleAV.add(new AttributeValue(person)); + peopleAV.add(AttributeValue.fromS(person)); ids.put(person, id); - idsAV.put(person, new AttributeValue().withN(Long.toString(id))); + idsAV.put(person, AttributeValue.fromN(Long.toString(id))); lists.put(person, people.subList(0, i + 1)); - listsAV.put(person, new AttributeValue().withL(Lists.newArrayList(peopleAV))); + listsAV.put(person, AttributeValue.fromL(Lists.newArrayList(peopleAV))); } Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(map)); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withM(idsAV)); - expectedItemMap.put(attributeNames.get(2), new AttributeValue().withM(listsAV)); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(map)); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromM(idsAV)); + expectedItemMap.put(attributeNames.get(2), AttributeValue.fromM(listsAV)); List rowData = Lists.newArrayList(); rowData.add(map); @@ -190,10 +190,10 @@ public void testMap() throws SerDeException { Map namesAV = Maps.newHashMap(); for (String person : people) { names.put(person, person); - namesAV.put(person, new AttributeValue(person)); + namesAV.put(person, AttributeValue.fromS(person)); } rowData.add(names); - expectedItemMap.put(attributeNames.get(3), new AttributeValue().withM(namesAV)); + expectedItemMap.put(attributeNames.get(3), AttributeValue.fromM(namesAV)); actualItemMap = getSerializedItem(attributeNames, colOIs, typeMapping, rowData); assertEquals(expectedItemMap, actualItemMap); @@ -207,14 +207,14 @@ public void testStruct() throws SerDeException { String struct = "animal"; List data = PRIMITIVE_STRING_DATA; Map dataAV = Maps.newHashMap(); - dataAV.put(PRIMITIVE_FIELDS.get(0), new AttributeValue(data.get(0))); - dataAV.put(PRIMITIVE_FIELDS.get(1), new AttributeValue().withN(data.get(1))); - dataAV.put(PRIMITIVE_FIELDS.get(2), new AttributeValue().withN(data.get(2))); - dataAV.put(PRIMITIVE_FIELDS.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + dataAV.put(PRIMITIVE_FIELDS.get(0), AttributeValue.fromS(data.get(0))); + dataAV.put(PRIMITIVE_FIELDS.get(1), AttributeValue.fromN(data.get(1))); + dataAV.put(PRIMITIVE_FIELDS.get(2), AttributeValue.fromN(data.get(2))); + dataAV.put(PRIMITIVE_FIELDS.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(struct)); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withM(dataAV)); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(struct)); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromM(dataAV)); List structData = Lists.newArrayList(); structData.add(data.get(0)); @@ -245,10 +245,10 @@ public void testItem() throws SerDeException { ); List data = PRIMITIVE_STRING_DATA; Map expectedItemMap = Maps.newHashMap(); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(data.get(0))); - expectedItemMap.put(attributeNames.get(1), new AttributeValue().withN(data.get(1))); - expectedItemMap.put(attributeNames.get(2), new AttributeValue().withN(data.get(2))); - expectedItemMap.put(attributeNames.get(3), new AttributeValue().withBOOL(Boolean.valueOf(data.get(3)))); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(data.get(0))); + expectedItemMap.put(attributeNames.get(1), AttributeValue.fromN(data.get(1))); + expectedItemMap.put(attributeNames.get(2), AttributeValue.fromN(data.get(2))); + expectedItemMap.put(attributeNames.get(3), AttributeValue.fromBool(Boolean.valueOf(data.get(3)))); Map itemCol = Maps.newHashMap(); for (int i = 0; i < attributeNames.size(); i++) { @@ -270,7 +270,7 @@ public void testItem() throws SerDeException { colNames.add(attributeNames.get(0)); colOIs.add(STRING_OBJECT_INSPECTOR); rowData.add(animal); - expectedItemMap.put(attributeNames.get(0), new AttributeValue(animal)); + expectedItemMap.put(attributeNames.get(0), AttributeValue.fromS(animal)); actualItemMap = getSerializedItem(colNames, colOIs, rowData); assertEquals(expectedItemMap, actualItemMap); diff --git a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandlerTest.java b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandlerTest.java index 4d46a5f3..4f50111b 100644 --- a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandlerTest.java +++ b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/DynamoDBStorageHandlerTest.java @@ -13,10 +13,6 @@ package org.apache.hadoop.hive.dynamodb; -import com.amazonaws.services.dynamodbv2.model.AttributeDefinition; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import org.apache.hadoop.dynamodb.DynamoDBConstants; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -32,6 +28,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class DynamoDBStorageHandlerTest { @@ -310,13 +310,20 @@ public void testCheckTableSchemaNullSerializationValid() throws MetaException { } private TableDescription getHashRangeTable() { - TableDescription description = new TableDescription().withKeySchema(Arrays.asList( - new KeySchemaElement().withAttributeName("hashKey"), - new KeySchemaElement().withAttributeName("rangeKey")) - ).withAttributeDefinitions(Arrays.asList( - new AttributeDefinition("hashKey", ScalarAttributeType.S), - new AttributeDefinition("rangeKey", ScalarAttributeType.N)) - ); + TableDescription description = TableDescription.builder() + .keySchema(Arrays.asList( + KeySchemaElement.builder().attributeName("hashKey").build(), + KeySchemaElement.builder().attributeName("rangeKey").build())) + .attributeDefinitions(Arrays.asList( + AttributeDefinition.builder() + .attributeName("hashKey") + .attributeType(ScalarAttributeType.S) + .build(), + AttributeDefinition.builder() + .attributeName("rangeKey") + .attributeType(ScalarAttributeType.N) + .build())) + .build(); return description; } } \ No newline at end of file diff --git a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdownTest.java b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdownTest.java index 957b15de..159d8cda 100644 --- a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdownTest.java +++ b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/filter/DynamoDBFilterPushdownTest.java @@ -1,12 +1,5 @@ package org.apache.hadoop.hive.dynamodb.filter; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; -import com.amazonaws.services.dynamodbv2.model.Condition; -import com.amazonaws.services.dynamodbv2.model.GlobalSecondaryIndexDescription; -import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; -import com.amazonaws.services.dynamodbv2.model.LocalSecondaryIndexDescription; -import com.amazonaws.services.dynamodbv2.model.Projection; -import com.amazonaws.services.dynamodbv2.model.ProjectionType; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.HashMap; @@ -26,6 +19,13 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.Condition; +import software.amazon.awssdk.services.dynamodb.model.GlobalSecondaryIndexDescription; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.LocalSecondaryIndexDescription; +import software.amazon.awssdk.services.dynamodb.model.Projection; +import software.amazon.awssdk.services.dynamodb.model.ProjectionType; public class DynamoDBFilterPushdownTest { @@ -445,9 +445,15 @@ private ExprNodeDesc buildPredicate(List predicates) { private List createKeySchema(String hashKeyName, String rangeKeyName) { List schema = new ArrayList<>(); - schema.add(new KeySchemaElement(hashKeyName, HASH_KEY_TYPE)); + schema.add(KeySchemaElement.builder() + .attributeName(hashKeyName) + .keyType(HASH_KEY_TYPE) + .build()); if (rangeKeyName != null) { - schema.add(new KeySchemaElement(rangeKeyName, RANGE_KEY_TYPE)); + schema.add(KeySchemaElement.builder() + .attributeName(rangeKeyName) + .keyType(RANGE_KEY_TYPE) + .build()); } return schema; } @@ -477,34 +483,39 @@ private Map initHiveTypeMapping() { private GlobalSecondaryIndexDescription createGSI(String indexName, String hashKeyName, String rangeKeyName, String projectionType, List nonKeyAttributes) { List schema = createKeySchema(hashKeyName, rangeKeyName); - Projection projection = new Projection() - .withProjectionType(projectionType) - .withNonKeyAttributes(nonKeyAttributes); - return new GlobalSecondaryIndexDescription() - .withIndexName(indexName) - .withKeySchema(schema) - .withProjection(projection); + Projection projection = Projection.builder() + .projectionType(projectionType) + .nonKeyAttributes(nonKeyAttributes) + .build(); + return GlobalSecondaryIndexDescription.builder() + .indexName(indexName) + .keySchema(schema) + .projection(projection) + .build(); } private LocalSecondaryIndexDescription createLSI(String indexName, String hashKeyName, String rangeKeyName, String projectionType, List nonKeyAttributes) { List schema = createKeySchema(hashKeyName, rangeKeyName); - Projection projection = new Projection() - .withProjectionType(projectionType) - .withNonKeyAttributes(nonKeyAttributes); - return new LocalSecondaryIndexDescription() - .withIndexName(indexName) - .withKeySchema(schema) - .withProjection(projection); + Projection projection = Projection.builder() + .projectionType(projectionType) + .nonKeyAttributes(nonKeyAttributes) + .build(); + return LocalSecondaryIndexDescription.builder() + .indexName(indexName) + .keySchema(schema) + .projection(projection) + .build(); } private void assertKeyCondition(String columnName, String columnValue, DynamoDBQueryFilter filter) { - Condition hashKeyCondition = new Condition(); List hashKeyAttributeValueList = new ArrayList<>(); - hashKeyAttributeValueList.add(new AttributeValue(columnValue)); - hashKeyCondition.setAttributeValueList(hashKeyAttributeValueList); - hashKeyCondition.setComparisonOperator(DynamoDBFilterOperator.EQ.getDynamoDBName()); + hashKeyAttributeValueList.add(AttributeValue.fromS(columnValue)); + Condition hashKeyCondition = Condition.builder() + .attributeValueList(hashKeyAttributeValueList) + .comparisonOperator(DynamoDBFilterOperator.EQ.getDynamoDBName()) + .build(); Assert.assertEquals((hashKeyCondition.toString()), filter.getKeyConditions().get(columnName).toString()); } diff --git a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBTypeTest.java b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBTypeTest.java index 43aa6cc0..6cffe5f0 100644 --- a/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBTypeTest.java +++ b/emr-dynamodb-hive/src/test/java/org/apache/hadoop/hive/dynamodb/type/HiveDynamoDBTypeTest.java @@ -11,7 +11,6 @@ package org.apache.hadoop.hive.dynamodb.type; -import com.amazonaws.services.dynamodbv2.model.AttributeValue; import org.apache.hadoop.dynamodb.test.DynamoDBTestUtils; import org.apache.hadoop.dynamodb.type.DynamoDBTypeConstants; import org.apache.hadoop.hive.serde2.SerDeException; @@ -35,6 +34,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import static org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyPrimitiveObjectInspectorFactory.LAZY_BOOLEAN_OBJECT_INSPECTOR; import static org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyPrimitiveObjectInspectorFactory.LAZY_DOUBLE_OBJECT_INSPECTOR; @@ -86,7 +86,7 @@ private static void initLazyObject(LazyObject lo, byte[] data, int start, int le public void testString() { String val = STRING_LIST.get(0); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(STRING_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withS(val); + AttributeValue expectedAV = AttributeValue.fromS(val); LazyString ls = new LazyString(LazyPrimitiveObjectInspectorFactory .getLazyStringObjectInspector(false, (byte) 0)); initLazyObject(ls, val.getBytes(), 0, val.length()); @@ -104,7 +104,7 @@ public void testDouble() { double val = TEST_DOUBLE; String valString = Double.toString(val); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(DOUBLE_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withN(valString); + AttributeValue expectedAV = AttributeValue.fromN(valString); AttributeValue actualAV = ddType.getDynamoDBData(val, DOUBLE_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualDouble = ddType.getHiveData(actualAV, DOUBLE_OBJECT_INSPECTOR); @@ -122,7 +122,7 @@ public void testLong() { long val = LONG_LIST.get(0); String valString = Long.toString(val); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(LONG_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withN(valString); + AttributeValue expectedAV = AttributeValue.fromN(valString); AttributeValue actualAV = ddType.getDynamoDBData(val, LONG_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualLong = ddType.getHiveData(actualAV, LONG_OBJECT_INSPECTOR); @@ -141,7 +141,7 @@ public void testBoolean() { boolean val = true; String valString = Boolean.toString(val); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(BOOLEAN_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withBOOL(val); + AttributeValue expectedAV = AttributeValue.fromBool(val); AttributeValue actualAV = ddType.getDynamoDBData(val, BOOLEAN_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualBoolean = ddType.getHiveData(actualAV, BOOLEAN_OBJECT_INSPECTOR); @@ -159,18 +159,18 @@ public void testList() { List longAVList = new ArrayList<>(); List strAVList = new ArrayList<>(); for (String str : STRING_LIST) { - longAVList.add(new AttributeValue().withN(str)); - strAVList.add(new AttributeValue(str)); + longAVList.add(AttributeValue.fromN(str)); + strAVList.add(AttributeValue.fromS(str)); } HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(LONG_LIST_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withL(longAVList); + AttributeValue expectedAV = AttributeValue.fromL(longAVList); AttributeValue actualAV = ddType.getDynamoDBData(LONG_LIST, LONG_LIST_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualList = ddType.getHiveData(actualAV, LONG_LIST_OBJECT_INSPECTOR); assertEquals(LONG_LIST, actualList); ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(STRING_LIST_OBJECT_INSPECTOR); - expectedAV = new AttributeValue().withL(strAVList); + expectedAV = AttributeValue.fromL(strAVList); actualAV = ddType.getDynamoDBData(STRING_LIST, STRING_LIST_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); actualList = ddType.getHiveData(actualAV, STRING_LIST_OBJECT_INSPECTOR); @@ -180,14 +180,14 @@ public void testList() { @Test public void testSet() { HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromDynamoDBType(DynamoDBTypeConstants.NUMBER_SET); - AttributeValue expectedAV = new AttributeValue().withNS(STRING_LIST); + AttributeValue expectedAV = AttributeValue.fromNs(STRING_LIST); AttributeValue actualAV = ddType.getDynamoDBData(LONG_LIST, LONG_LIST_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualList = ddType.getHiveData(actualAV, LONG_LIST_OBJECT_INSPECTOR); assertEquals(LONG_LIST, actualList); ddType = HiveDynamoDBTypeFactory.getTypeObjectFromDynamoDBType(DynamoDBTypeConstants.STRING_SET); - expectedAV = new AttributeValue().withSS(STRING_LIST); + expectedAV = AttributeValue.fromSs(STRING_LIST); actualAV = ddType.getDynamoDBData(STRING_LIST, STRING_LIST_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); actualList = ddType.getHiveData(actualAV, STRING_LIST_OBJECT_INSPECTOR); @@ -203,11 +203,11 @@ public void testItem() throws SerDeException { for (String str : STRING_MAP.keySet()) { String avStringField = DynamoDBTestUtils.toAttributeValueFieldFormat(DynamoDBTypeConstants.STRING); hiveStringItem.put(str, new JSONObject().put(avStringField, STRING_MAP.get(str)).toString()); - expectedStringItem.put(str, new AttributeValue(STRING_MAP.get(str))); + expectedStringItem.put(str, AttributeValue.fromS(STRING_MAP.get(str))); String avNumberField = DynamoDBTestUtils.toAttributeValueFieldFormat(DynamoDBTypeConstants.NUMBER); hiveNumberItem.put(str, new JSONObject().put(avNumberField, STRING_MAP.get(str)).toString()); - expectedNumberItem.put(str, new AttributeValue().withN(STRING_MAP.get(str))); + expectedNumberItem.put(str, AttributeValue.fromN(STRING_MAP.get(str))); } HiveDynamoDBItemType ddType = (HiveDynamoDBItemType) HiveDynamoDBTypeFactory @@ -228,18 +228,18 @@ public void testMap() { Map longAVMap = new HashMap<>(); Map stringAVMap = new HashMap<>(); for (String str : STRING_MAP.keySet()) { - longAVMap.put(str, new AttributeValue().withN(STRING_MAP.get(str))); - stringAVMap.put(str, new AttributeValue(STRING_MAP.get(str))); + longAVMap.put(str, AttributeValue.fromN(STRING_MAP.get(str))); + stringAVMap.put(str, AttributeValue.fromS(STRING_MAP.get(str))); } HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(LONG_MAP_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withM(longAVMap); + AttributeValue expectedAV = AttributeValue.fromM(longAVMap); AttributeValue actualAV = ddType.getDynamoDBData(LONG_MAP, LONG_MAP_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); Object actualMap = ddType.getHiveData(actualAV, LONG_MAP_OBJECT_INSPECTOR); assertEquals(LONG_MAP, actualMap); ddType = HiveDynamoDBTypeFactory.getTypeObjectFromDynamoDBType(DynamoDBTypeConstants.MAP); - expectedAV = new AttributeValue().withM(stringAVMap); + expectedAV = AttributeValue.fromM(stringAVMap); actualAV = ddType.getDynamoDBData(STRING_MAP, STRING_MAP_OBJECT_INSPECTOR, false); assertEquals(expectedAV, actualAV); actualMap = ddType.getHiveData(actualAV, STRING_MAP_OBJECT_INSPECTOR); @@ -250,9 +250,9 @@ public void testMap() { public void testStruct() { List struct = Lists.newArrayList((Object) STRING_LIST.get(0), LONG_LIST.get(1), TEST_DOUBLE); Map structAVMap = new HashMap<>(); - structAVMap.put(STRING_LIST.get(0), new AttributeValue(STRING_LIST.get(0))); - structAVMap.put(STRING_LIST.get(1), new AttributeValue().withN(Long.toString(LONG_LIST.get(1)))); - structAVMap.put(STRING_LIST.get(2), new AttributeValue().withN(Double.toString(TEST_DOUBLE))); + structAVMap.put(STRING_LIST.get(0), AttributeValue.fromS(STRING_LIST.get(0))); + structAVMap.put(STRING_LIST.get(1), AttributeValue.fromN(Long.toString(LONG_LIST.get(1)))); + structAVMap.put(STRING_LIST.get(2), AttributeValue.fromN(Double.toString(TEST_DOUBLE))); List structFieldNames = STRING_LIST.subList(0, 3); List structFieldOIs = Lists.newArrayList(STRING_OBJECT_INSPECTOR, LONG_OBJECT_INSPECTOR, DOUBLE_OBJECT_INSPECTOR); @@ -260,7 +260,7 @@ public void testStruct() { .getStandardStructObjectInspector(structFieldNames, structFieldOIs); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(structObjectInspector); - AttributeValue expectedAV = new AttributeValue().withM(structAVMap); + AttributeValue expectedAV = AttributeValue.fromM(structAVMap); AttributeValue actualAV = ddType.getDynamoDBData(struct, structObjectInspector, false); assertEquals(expectedAV, actualAV); Object actualStruct = ddType.getHiveData(actualAV, structObjectInspector); @@ -270,9 +270,9 @@ public void testStruct() { @Test public void testMultipleTypeList() { List avList = new ArrayList<>(); - avList.add(new AttributeValue(STRING_LIST.get(0))); - avList.add(new AttributeValue().withN(STRING_LIST.get(0))); - AttributeValue av = new AttributeValue().withL(avList); + avList.add(AttributeValue.fromS(STRING_LIST.get(0))); + avList.add(AttributeValue.fromN(STRING_LIST.get(0))); + AttributeValue av = AttributeValue.fromL(avList); HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(STRING_LIST_OBJECT_INSPECTOR); List expectedStringList = Lists.newArrayList(STRING_LIST.get(0), null); @@ -289,7 +289,7 @@ public void testMultipleTypeList() { public void testNullSerialization() { String val = null; HiveDynamoDBType ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(STRING_OBJECT_INSPECTOR); - AttributeValue expectedAV = new AttributeValue().withNULL(true); + AttributeValue expectedAV = AttributeValue.fromNul(true); AttributeValue actualAV = ddType.getDynamoDBData(val, STRING_OBJECT_INSPECTOR, true); assertEquals(expectedAV, actualAV); Object actualNull = ddType.getHiveData(actualAV, STRING_OBJECT_INSPECTOR); @@ -298,13 +298,13 @@ public void testNullSerialization() { List nullStringList = new ArrayList<>(STRING_LIST); nullStringList.set(0, null); List strAVList = new ArrayList<>(); - strAVList.add(new AttributeValue().withNULL(true)); + strAVList.add(AttributeValue.fromNul(true)); for (int i = 1; i < nullStringList.size(); i++) { - strAVList.add(new AttributeValue(nullStringList.get(i))); + strAVList.add(AttributeValue.fromS(nullStringList.get(i))); } ddType = HiveDynamoDBTypeFactory.getTypeObjectFromHiveType(STRING_LIST_OBJECT_INSPECTOR); - expectedAV = new AttributeValue().withL(strAVList); + expectedAV = AttributeValue.fromL(strAVList); actualAV = ddType.getDynamoDBData(nullStringList, STRING_LIST_OBJECT_INSPECTOR, true); assertEquals(expectedAV, actualAV); Object actualList = ddType.getHiveData(actualAV, STRING_LIST_OBJECT_INSPECTOR); diff --git a/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBExport.java b/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBExport.java index 5ff30bbd..8058f090 100644 --- a/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBExport.java +++ b/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBExport.java @@ -13,7 +13,6 @@ package org.apache.hadoop.dynamodb.tools; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import java.util.Date; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,6 +31,8 @@ import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class DynamoDBExport extends Configured implements Tool { @@ -106,16 +107,16 @@ private void setTableProperties(JobConf jobConf, String tableName, Double readRa DynamoDBClient client = new DynamoDBClient(jobConf); TableDescription description = client.describeTable(tableName); - Long itemCount = description.getItemCount(); - Long tableSizeBytes = description.getTableSizeBytes(); + Long itemCount = description.itemCount(); + Long tableSizeBytes = description.tableSizeBytes(); - if (description.getBillingModeSummary() == null - || description.getBillingModeSummary().getBillingMode() - .equals(DynamoDBConstants.BILLING_MODE_PROVISIONED)) { + if (description.billingModeSummary() == null + || description.billingModeSummary().billingMode() + .equals(BillingMode.PROVISIONED)) { jobConf.set(DynamoDBConstants.READ_THROUGHPUT, - description.getProvisionedThroughput().getReadCapacityUnits().toString()); + description.provisionedThroughput().readCapacityUnits().toString()); jobConf.set(DynamoDBConstants.WRITE_THROUGHPUT, - description.getProvisionedThroughput().getWriteCapacityUnits().toString()); + description.provisionedThroughput().writeCapacityUnits().toString()); } else { // If not specified at the table level, set a hard coded value of 40,000 jobConf.set(DynamoDBConstants.READ_THROUGHPUT, diff --git a/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBImport.java b/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBImport.java index 6ff320b8..be5e6c05 100644 --- a/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBImport.java +++ b/emr-dynamodb-tools/src/main/java/org/apache/hadoop/dynamodb/tools/DynamoDBImport.java @@ -13,7 +13,6 @@ package org.apache.hadoop.dynamodb.tools; -import com.amazonaws.services.dynamodbv2.model.TableDescription; import java.util.Date; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,6 +31,8 @@ import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; public class DynamoDBImport extends Configured implements Tool { @@ -93,13 +94,12 @@ private void setTableProperties(JobConf jobConf, String tableName, Double writeR DynamoDBClient client = new DynamoDBClient(jobConf); TableDescription description = client.describeTable(tableName); - if (description.getBillingModeSummary() == null - || description.getBillingModeSummary().getBillingMode() - .equals(DynamoDBConstants.BILLING_MODE_PROVISIONED)) { + if (description.billingModeSummary() == null + || description.billingModeSummary().billingMode() == BillingMode.PROVISIONED) { jobConf.set(DynamoDBConstants.READ_THROUGHPUT, - description.getProvisionedThroughput().getReadCapacityUnits().toString()); + description.provisionedThroughput().readCapacityUnits().toString()); jobConf.set(DynamoDBConstants.WRITE_THROUGHPUT, - description.getProvisionedThroughput().getWriteCapacityUnits().toString()); + description.provisionedThroughput().writeCapacityUnits().toString()); } else { jobConf.set(DynamoDBConstants.READ_THROUGHPUT, DynamoDBConstants.DEFAULT_CAPACITY_FOR_ON_DEMAND.toString()); diff --git a/pom.xml b/pom.xml index 2c6f6c03..1a0185a9 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,7 @@ 1.8 - 1.11.475 + 2.18.40 3.7 3.3.3 1.0.0 @@ -68,9 +68,11 @@ - com.amazonaws - aws-java-sdk-dynamodb + software.amazon.awssdk + bom ${aws-java-sdk.version} + pom + import com.google.guava