diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index f2d08a02a760..4d3c6be4a01b 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -27,16 +27,6 @@ - - com.amazonaws - aws-java-sdk-core - - - - com.amazonaws - aws-java-sdk-glue - - com.fasterxml.jackson.core jackson-core @@ -194,6 +184,31 @@ jmxutils + + software.amazon.awssdk + auth + + + + software.amazon.awssdk + glue + + + + software.amazon.awssdk + metrics-spi + + + + software.amazon.awssdk + sdk-core + + + + software.amazon.awssdk + utils + + com.fasterxml.jackson.core jackson-annotations @@ -447,7 +462,16 @@ aws-java-sdk-core --> com.amazonaws:aws-java-sdk-core org.apache.iceberg:iceberg-api + software.amazon.awssdk:auth + software.amazon.awssdk:metrics-spi + software.amazon.awssdk:sdk-core + + + software.amazon.awssdk:utils + diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreModule.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreModule.java index fa400129eea6..450f68930ec8 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreModule.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreModule.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.deltalake.metastore.glue; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.TypeLiteral; @@ -21,6 +20,7 @@ import io.trino.plugin.deltalake.AllowDeltaLakeManagedTableRename; import io.trino.plugin.hive.metastore.glue.ForGlueHiveMetastore; import io.trino.plugin.hive.metastore.glue.GlueMetastoreModule; +import software.amazon.awssdk.services.glue.model.Table; import java.util.function.Predicate; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreTableFilterProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreTableFilterProvider.java index dce37451861e..bb8f682d846e 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreTableFilterProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/DeltaLakeGlueMetastoreTableFilterProvider.java @@ -13,10 +13,10 @@ */ package io.trino.plugin.deltalake.metastore.glue; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Inject; import com.google.inject.Provider; import io.trino.plugin.hive.metastore.glue.DefaultGlueMetastoreTableFilterProvider; +import software.amazon.awssdk.services.glue.model.Table; import java.util.function.Predicate; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeCleanUpGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeCleanUpGlueMetastore.java index d6716dd463d3..228e3de8c92b 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeCleanUpGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeCleanUpGlueMetastore.java @@ -13,20 +13,19 @@ */ package io.trino.plugin.deltalake.metastore.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetDatabasesResult; +import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; import io.trino.plugin.hive.aws.AwsApiCallStats; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; import java.util.List; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncPaginatedRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static java.lang.System.currentTimeMillis; import static java.util.concurrent.TimeUnit.DAYS; @@ -39,28 +38,27 @@ public class TestDeltaLakeCleanUpGlueMetastore @Test public void cleanupOrphanedDatabases() { - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); + GlueAsyncClient glueClient = GlueAsyncClient.builder().build(); long creationTimeMillisThreshold = currentTimeMillis() - DAYS.toMillis(1); - List orphanedDatabases = getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - new AwsApiCallStats()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .filter(glueDatabase -> glueDatabase.getName().startsWith(TEST_DATABASE_NAME_PREFIX) && - glueDatabase.getCreateTime().getTime() <= creationTimeMillisThreshold) - .map(com.amazonaws.services.glue.model.Database::getName) - .collect(toImmutableList()); + ImmutableList.Builder databaseNames = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getDatabasesPaginator(GetDatabasesRequest.builder().build()), + getDatabasesResponse -> getDatabasesResponse.databaseList() + .stream() + .filter(glueDatabase -> glueDatabase.name().startsWith(TEST_DATABASE_NAME_PREFIX) + && glueDatabase.createTime().toEpochMilli() <= creationTimeMillisThreshold) + .forEach(glueDatabase -> { + databaseNames.add(glueDatabase.name()); + }), + new AwsApiCallStats()); + List orphanedDatabases = databaseNames.build(); if (!orphanedDatabases.isEmpty()) { log.info("Found %s %s* databases that look orphaned, removing", orphanedDatabases.size(), TEST_DATABASE_NAME_PREFIX); orphanedDatabases.forEach(database -> { try { log.info("Deleting %s database", database); - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(database)); + awsSyncRequest(glueClient::deleteDatabase, DeleteDatabaseRequest.builder() + .name(database).build(), null); } catch (EntityNotFoundException e) { log.info("Database [%s] not found, could be removed by other cleanup process", database); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java index f4aaad48804b..49e4f891fa8d 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java @@ -13,9 +13,6 @@ */ package io.trino.plugin.deltalake.metastore.glue; -import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.ConcurrentModificationException; import io.trino.Session; import io.trino.plugin.deltalake.TestingDeltaLakePlugin; import io.trino.plugin.deltalake.metastore.TestingDeltaLakeMetastoreModule; @@ -29,6 +26,9 @@ import io.trino.testing.QueryRunner; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.ConcurrentModificationException; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -74,14 +74,14 @@ protected QueryRunner createQueryRunner() GlueHiveMetastoreConfig glueConfig = new GlueHiveMetastoreConfig() .setDefaultWarehouseDir(dataDirectory.toUri().toString()); - AWSGlueAsync glueClient = createAsyncGlueClient(glueConfig, DefaultAWSCredentialsProviderChain.getInstance(), Optional.empty(), stats.newRequestMetricsCollector()); - AWSGlueAsync proxiedGlueClient = newProxy(AWSGlueAsync.class, (proxy, method, args) -> { + GlueAsyncClient glueClient = createAsyncGlueClient(glueConfig, DefaultCredentialsProvider.create(), Optional.empty(), stats.newRequestMetricsPublisher()); + GlueAsyncClient proxiedGlueClient = newProxy(GlueAsyncClient.class, (proxy, method, args) -> { Object result; try { if (method.getName().equals("deleteTable") && failNextGlueDeleteTableCall.get()) { // Simulate concurrent modifications on the table that is about to be dropped failNextGlueDeleteTableCall.set(false); - throw new TrinoException(HIVE_METASTORE_ERROR, new ConcurrentModificationException("Test-simulated metastore concurrent modification exception")); + throw new TrinoException(HIVE_METASTORE_ERROR, ConcurrentModificationException.builder().message("Test-simulated metastore concurrent modification exception").build()); } result = method.invoke(glueClient, args); } diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index bd19a453a590..5bd4ce356d27 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -31,11 +31,6 @@ aws-java-sdk-core - - com.amazonaws - aws-java-sdk-glue - - com.amazonaws aws-java-sdk-s3 @@ -272,6 +267,67 @@ jmxutils + + software.amazon.awssdk + apache-client + + + commons-logging + commons-logging + + + + + + software.amazon.awssdk + auth + + + + software.amazon.awssdk + aws-core + + + + software.amazon.awssdk + glue + + + + software.amazon.awssdk + http-client-spi + + + + software.amazon.awssdk + metrics-spi + + + + software.amazon.awssdk + netty-nio-client + + + + software.amazon.awssdk + regions + + + + software.amazon.awssdk + sdk-core + + + + software.amazon.awssdk + sts + + + + software.amazon.awssdk + utils + + com.fasterxml.jackson.core jackson-annotations @@ -481,6 +537,19 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + + software.amazon.awssdk:utils + + + + org.apache.maven.plugins maven-surefire-plugin diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/AwsSdkUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/AwsSdkUtil.java index 646252513c11..3e5c417eea7a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/AwsSdkUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/AwsSdkUtil.java @@ -15,26 +15,27 @@ import com.google.common.collect.AbstractIterator; import io.trino.plugin.hive.aws.AwsApiCallStats; +import io.trino.spi.TrinoException; +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.services.glue.model.GlueException; import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Stream; import static com.google.common.collect.Streams.stream; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static java.util.Objects.requireNonNull; public final class AwsSdkUtil { private AwsSdkUtil() {} - /** - * Helper method to get all results from a paginated API. - * - * @param request request object reused for subsequent requests with - * {@code setNextToken} being used to set the next token in the request object - */ - public static Stream getPaginatedResults( + public static Stream getPaginatedResultsForS3( Function submission, Request request, BiConsumer setNextToken, @@ -68,4 +69,56 @@ protected Result computeNext() return stream(iterator); } + + /** + * Helper method to handle sync request with async client + */ + public static Result awsSyncRequest( + Function> submission, + Request request, + AwsApiCallStats stats) + { + requireNonNull(submission, "submission is null"); + requireNonNull(request, "request is null"); + try { + if (stats != null) { + return stats.call(() -> submission.apply(request).join()); + } + + return submission.apply(request).join(); + } + catch (CompletionException e) { + if (e.getCause() instanceof GlueException glueException) { + throw glueException; + } + throw new TrinoException(HIVE_METASTORE_ERROR, e.getCause()); + } + } + + /** + * Helper method to handle sync paginated request with async client + */ + public static void awsSyncPaginatedRequest(SdkPublisher paginator, Consumer resultConsumer, AwsApiCallStats stats) + { + requireNonNull(paginator, "paginator is null"); + requireNonNull(resultConsumer, "resultConsumer is null"); + try { + CompletableFuture paginationFuture; + if (stats != null) { + paginationFuture = stats.call(() -> paginator.subscribe(resultConsumer)); + } + else { + paginationFuture = paginator.subscribe(resultConsumer); + } + + // for paginator.subscribe() to complete + paginationFuture.join(); + } + catch (CompletionException e) { + if (e.getCause() instanceof GlueException glueException) { + throw glueException; + } + throw new TrinoException(HIVE_METASTORE_ERROR, e.getCause()); + } + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProvider.java index 92e85f14e748..0caad4a21266 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProvider.java @@ -13,23 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.ColumnStatistics; -import com.amazonaws.services.glue.model.ColumnStatisticsData; -import com.amazonaws.services.glue.model.ColumnStatisticsType; -import com.amazonaws.services.glue.model.DateColumnStatisticsData; -import com.amazonaws.services.glue.model.DecimalColumnStatisticsData; -import com.amazonaws.services.glue.model.DeleteColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.DeleteColumnStatisticsForTableRequest; -import com.amazonaws.services.glue.model.DoubleColumnStatisticsData; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionResult; -import com.amazonaws.services.glue.model.GetColumnStatisticsForTableRequest; -import com.amazonaws.services.glue.model.GetColumnStatisticsForTableResult; -import com.amazonaws.services.glue.model.LongColumnStatisticsData; -import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableRequest; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -42,6 +25,23 @@ import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreUtil; import io.trino.spi.TrinoException; import io.trino.spi.type.Type; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.ColumnStatistics; +import software.amazon.awssdk.services.glue.model.ColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.ColumnStatisticsType; +import software.amazon.awssdk.services.glue.model.DateColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.DecimalColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.DeleteColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.DeleteColumnStatisticsForTableRequest; +import software.amazon.awssdk.services.glue.model.DoubleColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForPartitionResponse; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForTableRequest; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForTableResponse; +import software.amazon.awssdk.services.glue.model.LongColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.UpdateColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdateColumnStatisticsForTableRequest; import java.util.ArrayList; import java.util.Collection; @@ -57,6 +57,7 @@ import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_NOT_FOUND; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.plugin.hive.metastore.glue.converter.GlueStatConverter.fromGlueColumnStatistics; import static io.trino.plugin.hive.metastore.glue.converter.GlueStatConverter.toGlueColumnStatistics; import static io.trino.plugin.hive.metastore.thrift.ThriftMetastoreUtil.getHiveBasicStatistics; @@ -77,11 +78,11 @@ public class DefaultGlueColumnStatisticsProvider private static final int GLUE_COLUMN_WRITE_STAT_PAGE_SIZE = 25; private final GlueMetastoreStats stats; - private final AWSGlueAsync glueClient; + private final GlueAsyncClient glueClient; private final Executor readExecutor; private final Executor writeExecutor; - public DefaultGlueColumnStatisticsProvider(AWSGlueAsync glueClient, Executor readExecutor, Executor writeExecutor, GlueMetastoreStats stats) + public DefaultGlueColumnStatisticsProvider(GlueAsyncClient glueClient, Executor readExecutor, Executor writeExecutor, GlueMetastoreStats stats) { this.glueClient = glueClient; this.readExecutor = readExecutor; @@ -101,23 +102,23 @@ public Map getTableColumnStatistics(Table table) try { List columnNames = getAllColumns(table); List> columnChunks = Lists.partition(columnNames, GLUE_COLUMN_READ_STAT_PAGE_SIZE); - List> getStatsFutures = columnChunks.stream() + List> getStatsFutures = columnChunks.stream() .map(partialColumns -> supplyAsync(() -> { - GetColumnStatisticsForTableRequest request = new GetColumnStatisticsForTableRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withColumnNames(partialColumns); - return stats.getGetColumnStatisticsForTable().call(() -> glueClient.getColumnStatisticsForTable(request)); + GetColumnStatisticsForTableRequest request = GetColumnStatisticsForTableRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .columnNames(partialColumns).build(); + return awsSyncRequest(glueClient::getColumnStatisticsForTable, request, stats.getGetColumnStatisticsForTable()); }, readExecutor)).collect(toImmutableList()); HiveBasicStatistics tableStatistics = getHiveBasicStatistics(table.getParameters()); ImmutableMap.Builder columnStatsMapBuilder = ImmutableMap.builder(); - for (CompletableFuture future : getStatsFutures) { - GetColumnStatisticsForTableResult tableColumnsStats = getFutureValue(future, TrinoException.class); - for (ColumnStatistics columnStatistics : tableColumnsStats.getColumnStatisticsList()) { + for (CompletableFuture future : getStatsFutures) { + GetColumnStatisticsForTableResponse tableColumnsStats = getFutureValue(future, TrinoException.class); + for (ColumnStatistics columnStatistics : tableColumnsStats.columnStatisticsList()) { columnStatsMapBuilder.put( - columnStatistics.getColumnName(), - fromGlueColumnStatistics(columnStatistics.getStatisticsData(), tableStatistics.getRowCount())); + columnStatistics.columnName(), + fromGlueColumnStatistics(columnStatistics.statisticsData(), tableStatistics.getRowCount())); } } return columnStatsMapBuilder.buildOrThrow(); @@ -130,20 +131,20 @@ public Map getTableColumnStatistics(Table table) @Override public Map> getPartitionColumnStatistics(Collection partitions) { - Map>> resultsForPartition = new HashMap<>(); + Map>> resultsForPartition = new HashMap<>(); for (Partition partition : partitions) { - ImmutableList.Builder> futures = ImmutableList.builder(); + ImmutableList.Builder> futures = ImmutableList.builder(); List> columnChunks = Lists.partition(partition.getColumns(), GLUE_COLUMN_READ_STAT_PAGE_SIZE); for (List partialPartitionColumns : columnChunks) { List columnsNames = partialPartitionColumns.stream() .map(Column::getName) .collect(toImmutableList()); - GetColumnStatisticsForPartitionRequest request = new GetColumnStatisticsForPartitionRequest() - .withDatabaseName(partition.getDatabaseName()) - .withTableName(partition.getTableName()) - .withColumnNames(columnsNames) - .withPartitionValues(partition.getValues()); - futures.add(supplyAsync(() -> stats.getGetColumnStatisticsForPartition().call(() -> glueClient.getColumnStatisticsForPartition(request)), readExecutor)); + GetColumnStatisticsForPartitionRequest request = GetColumnStatisticsForPartitionRequest.builder() + .databaseName(partition.getDatabaseName()) + .tableName(partition.getTableName()) + .columnNames(columnsNames) + .partitionValues(partition.getValues()).build(); + futures.add(supplyAsync(() -> awsSyncRequest(glueClient::getColumnStatisticsForPartition, request, stats.getGetColumnStatisticsForPartition()), readExecutor)); } resultsForPartition.put(partition, futures.build()); } @@ -154,12 +155,12 @@ public Map> getPartitionColumnStati HiveBasicStatistics tableStatistics = getHiveBasicStatistics(partition.getParameters()); ImmutableMap.Builder columnStatsMapBuilder = ImmutableMap.builder(); - for (CompletableFuture getColumnStatisticsResultFuture : futures) { - GetColumnStatisticsForPartitionResult getColumnStatisticsResult = getFutureValue(getColumnStatisticsResultFuture); - getColumnStatisticsResult.getColumnStatisticsList().forEach(columnStatistics -> + for (CompletableFuture getColumnStatisticsResponseFuture : futures) { + GetColumnStatisticsForPartitionResponse getColumnStatisticsResult = getFutureValue(getColumnStatisticsResponseFuture); + getColumnStatisticsResult.columnStatisticsList().forEach(columnStatistics -> columnStatsMapBuilder.put( - columnStatistics.getColumnName(), - fromGlueColumnStatistics(columnStatistics.getStatisticsData(), tableStatistics.getRowCount()))); + columnStatistics.columnName(), + fromGlueColumnStatistics(columnStatistics.statisticsData(), tableStatistics.getRowCount()))); } partitionStatistics.put(partition, columnStatsMapBuilder.buildOrThrow()); @@ -180,23 +181,23 @@ public Map> getPartitionColumnStati // this can be removed once glue fix this behaviour private boolean isGlueWritable(ColumnStatistics stats) { - ColumnStatisticsData statisticsData = stats.getStatisticsData(); - String columnType = stats.getStatisticsData().getType(); + ColumnStatisticsData statisticsData = stats.statisticsData(); + String columnType = stats.statisticsData().typeAsString(); if (columnType.equals(ColumnStatisticsType.DATE.toString())) { - DateColumnStatisticsData data = statisticsData.getDateColumnStatisticsData(); - return data.getMaximumValue() != null && data.getMinimumValue() != null; + DateColumnStatisticsData data = statisticsData.dateColumnStatisticsData(); + return data.maximumValue() != null && data.minimumValue() != null; } if (columnType.equals(ColumnStatisticsType.DECIMAL.toString())) { - DecimalColumnStatisticsData data = statisticsData.getDecimalColumnStatisticsData(); - return data.getMaximumValue() != null && data.getMinimumValue() != null; + DecimalColumnStatisticsData data = statisticsData.decimalColumnStatisticsData(); + return data.maximumValue() != null && data.minimumValue() != null; } if (columnType.equals(ColumnStatisticsType.DOUBLE.toString())) { - DoubleColumnStatisticsData data = statisticsData.getDoubleColumnStatisticsData(); - return data.getMaximumValue() != null && data.getMinimumValue() != null; + DoubleColumnStatisticsData data = statisticsData.doubleColumnStatisticsData(); + return data.maximumValue() != null && data.minimumValue() != null; } if (columnType.equals(ColumnStatisticsType.LONG.toString())) { - LongColumnStatisticsData data = statisticsData.getLongColumnStatisticsData(); - return data.getMaximumValue() != null && data.getMinimumValue() != null; + LongColumnStatisticsData data = statisticsData.longColumnStatisticsData(); + return data.maximumValue() != null && data.minimumValue() != null; } return true; } @@ -213,22 +214,19 @@ public void updateTableColumnStatistics(Table table, Map> columnChunks = Lists.partition(columnStats, GLUE_COLUMN_WRITE_STAT_PAGE_SIZE); List> updateFutures = columnChunks.stream().map(columnChunk -> runAsync( - () -> stats.getUpdateColumnStatisticsForTable().call(() -> glueClient.updateColumnStatisticsForTable( - new UpdateColumnStatisticsForTableRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withColumnStatisticsList(columnChunk))), this.writeExecutor)) + () -> awsSyncRequest(glueClient::updateColumnStatisticsForTable, UpdateColumnStatisticsForTableRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .columnStatisticsList(columnChunk).build(), stats.getUpdateColumnStatisticsForTable()), this.writeExecutor)) .collect(toUnmodifiableList()); Map currentTableColumnStatistics = this.getTableColumnStatistics(table); Set removedStatistics = difference(currentTableColumnStatistics.keySet(), updatedTableColumnStatistics.keySet()); List> deleteFutures = removedStatistics.stream() - .map(column -> runAsync(() -> stats.getDeleteColumnStatisticsForTable().call(() -> - glueClient.deleteColumnStatisticsForTable( - new DeleteColumnStatisticsForTableRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withColumnName(column))), this.writeExecutor)) + .map(column -> runAsync(() -> awsSyncRequest(glueClient::deleteColumnStatisticsForTable, DeleteColumnStatisticsForTableRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .columnName(column).build(), stats.getDeleteColumnStatisticsForTable()), this.writeExecutor)) .collect(toUnmodifiableList()); ImmutableList> updateOperationsFutures = ImmutableList.>builder() @@ -262,24 +260,20 @@ public void updatePartitionStatistics(Set partitionSt List> columnChunks = Lists.partition(columnStats, GLUE_COLUMN_WRITE_STAT_PAGE_SIZE); columnChunks.forEach(columnChunk -> - updateFutures.add(runAsync(() -> stats.getUpdateColumnStatisticsForPartition().call(() -> - glueClient.updateColumnStatisticsForPartition( - new UpdateColumnStatisticsForPartitionRequest() - .withDatabaseName(partition.getDatabaseName()) - .withTableName(partition.getTableName()) - .withPartitionValues(partition.getValues()) - .withColumnStatisticsList(columnChunk))), + updateFutures.add(runAsync(() -> awsSyncRequest(glueClient::updateColumnStatisticsForPartition, UpdateColumnStatisticsForPartitionRequest.builder() + .databaseName(partition.getDatabaseName()) + .tableName(partition.getTableName()) + .partitionValues(partition.getValues()) + .columnStatisticsList(columnChunk).build(), stats.getUpdateColumnStatisticsForPartition()), writeExecutor))); Set removedStatistics = difference(currentStatistics.get(partition).keySet(), updatedColumnStatistics.keySet()); removedStatistics.forEach(column -> - updateFutures.add(runAsync(() -> stats.getDeleteColumnStatisticsForPartition().call(() -> - glueClient.deleteColumnStatisticsForPartition( - new DeleteColumnStatisticsForPartitionRequest() - .withDatabaseName(partition.getDatabaseName()) - .withTableName(partition.getTableName()) - .withPartitionValues(partition.getValues()) - .withColumnName(column))), + updateFutures.add(runAsync(() -> awsSyncRequest(glueClient::deleteColumnStatisticsForPartition, DeleteColumnStatisticsForPartitionRequest.builder() + .databaseName(partition.getDatabaseName()) + .tableName(partition.getTableName()) + .partitionValues(partition.getValues()) + .columnName(column).build(), stats.getDeleteColumnStatisticsForPartition()), writeExecutor))); } try { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProviderFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProviderFactory.java index c08a3f545aa0..39a6a28d2fc7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProviderFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueColumnStatisticsProviderFactory.java @@ -13,8 +13,8 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.inject.Inject; +import software.amazon.awssdk.services.glue.GlueAsyncClient; import java.util.concurrent.Executor; @@ -36,7 +36,7 @@ public DefaultGlueColumnStatisticsProviderFactory( } @Override - public GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(AWSGlueAsync glueClient, GlueMetastoreStats stats) + public GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(GlueAsyncClient glueClient, GlueMetastoreStats stats) { return new DefaultGlueColumnStatisticsProvider( glueClient, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueMetastoreTableFilterProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueMetastoreTableFilterProvider.java index cd3e3b5324de..e6057df4bdcb 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueMetastoreTableFilterProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DefaultGlueMetastoreTableFilterProvider.java @@ -13,10 +13,10 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Inject; import com.google.inject.Provider; import io.trino.plugin.hive.HideDeltaLakeTables; +import software.amazon.awssdk.services.glue.model.Table; import java.util.Map; import java.util.function.Predicate; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DisabledGlueColumnStatisticsProviderFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DisabledGlueColumnStatisticsProviderFactory.java index 6a06aa5bc33f..f04eb21610d7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DisabledGlueColumnStatisticsProviderFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/DisabledGlueColumnStatisticsProviderFactory.java @@ -13,13 +13,13 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.AWSGlueAsync; +import software.amazon.awssdk.services.glue.GlueAsyncClient; public class DisabledGlueColumnStatisticsProviderFactory implements GlueColumnStatisticsProviderFactory { @Override - public GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(AWSGlueAsync glueClient, GlueMetastoreStats stats) + public GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(GlueAsyncClient glueClient, GlueMetastoreStats stats) { return new DisabledGlueColumnStatisticsProvider(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCatalogIdRequestHandler.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCatalogIdRequestHandler.java index bea84a417160..97b6a29e4b3e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCatalogIdRequestHandler.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCatalogIdRequestHandler.java @@ -13,36 +13,38 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.model.BatchCreatePartitionRequest; -import com.amazonaws.services.glue.model.BatchGetPartitionRequest; -import com.amazonaws.services.glue.model.BatchUpdatePartitionRequest; -import com.amazonaws.services.glue.model.CreateDatabaseRequest; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.DeleteColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.DeleteColumnStatisticsForTableRequest; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.DeletePartitionRequest; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.GetColumnStatisticsForTableRequest; -import com.amazonaws.services.glue.model.GetDatabaseRequest; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetPartitionRequest; -import com.amazonaws.services.glue.model.GetPartitionsRequest; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.GetTablesRequest; -import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionRequest; -import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableRequest; -import com.amazonaws.services.glue.model.UpdateDatabaseRequest; -import com.amazonaws.services.glue.model.UpdatePartitionRequest; -import com.amazonaws.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.model.BatchCreatePartitionRequest; +import software.amazon.awssdk.services.glue.model.BatchGetPartitionRequest; +import software.amazon.awssdk.services.glue.model.BatchUpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.DeleteColumnStatisticsForTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeletePartitionRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetColumnStatisticsForTableRequest; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionsRequest; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.UpdateColumnStatisticsForPartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdateColumnStatisticsForTableRequest; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; import static java.util.Objects.requireNonNull; public class GlueCatalogIdRequestHandler - extends RequestHandler2 + implements ExecutionInterceptor { private final String catalogId; @@ -52,76 +54,77 @@ public GlueCatalogIdRequestHandler(String catalogId) } @Override - public AmazonWebServiceRequest beforeExecution(AmazonWebServiceRequest request) + public SdkRequest modifyRequest(Context.ModifyRequest context, ExecutionAttributes executionAttributes) { + SdkRequest request = context.request(); if (request instanceof GetDatabasesRequest) { - return ((GetDatabasesRequest) request).withCatalogId(catalogId); + return ((GetDatabasesRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetDatabaseRequest) { - return ((GetDatabaseRequest) request).withCatalogId(catalogId); + return ((GetDatabaseRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof CreateDatabaseRequest) { - return ((CreateDatabaseRequest) request).withCatalogId(catalogId); + return ((CreateDatabaseRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof UpdateDatabaseRequest) { - return ((UpdateDatabaseRequest) request).withCatalogId(catalogId); + return ((UpdateDatabaseRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof DeleteDatabaseRequest) { - return ((DeleteDatabaseRequest) request).withCatalogId(catalogId); + return ((DeleteDatabaseRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetTablesRequest) { - return ((GetTablesRequest) request).withCatalogId(catalogId); + return ((GetTablesRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetTableRequest) { - return ((GetTableRequest) request).withCatalogId(catalogId); + return ((GetTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof CreateTableRequest) { - return ((CreateTableRequest) request).withCatalogId(catalogId); + return ((CreateTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof UpdateTableRequest) { - return ((UpdateTableRequest) request).withCatalogId(catalogId); + return ((UpdateTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof DeleteTableRequest) { - return ((DeleteTableRequest) request).withCatalogId(catalogId); + return ((DeleteTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetPartitionsRequest) { - return ((GetPartitionsRequest) request).withCatalogId(catalogId); + return ((GetPartitionsRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetPartitionRequest) { - return ((GetPartitionRequest) request).withCatalogId(catalogId); + return ((GetPartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof UpdatePartitionRequest) { - return ((UpdatePartitionRequest) request).withCatalogId(catalogId); + return ((UpdatePartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof DeletePartitionRequest) { - return ((DeletePartitionRequest) request).withCatalogId(catalogId); + return ((DeletePartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof BatchGetPartitionRequest) { - return ((BatchGetPartitionRequest) request).withCatalogId(catalogId); + return ((BatchGetPartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof BatchCreatePartitionRequest) { - return ((BatchCreatePartitionRequest) request).withCatalogId(catalogId); + return ((BatchCreatePartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof BatchUpdatePartitionRequest) { - return ((BatchUpdatePartitionRequest) request).withCatalogId(catalogId); + return ((BatchUpdatePartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetColumnStatisticsForTableRequest) { - return ((GetColumnStatisticsForTableRequest) request).withCatalogId(catalogId); + return ((GetColumnStatisticsForTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof UpdateColumnStatisticsForTableRequest) { - return ((UpdateColumnStatisticsForTableRequest) request).withCatalogId(catalogId); + return ((UpdateColumnStatisticsForTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof DeleteColumnStatisticsForTableRequest) { - return ((DeleteColumnStatisticsForTableRequest) request).withCatalogId(catalogId); + return ((DeleteColumnStatisticsForTableRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof GetColumnStatisticsForPartitionRequest) { - return ((GetColumnStatisticsForPartitionRequest) request).withCatalogId(catalogId); + return ((GetColumnStatisticsForPartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof UpdateColumnStatisticsForPartitionRequest) { - return ((UpdateColumnStatisticsForPartitionRequest) request).withCatalogId(catalogId); + return ((UpdateColumnStatisticsForPartitionRequest) request).toBuilder().catalogId(catalogId).build(); } if (request instanceof DeleteColumnStatisticsForPartitionRequest) { - return ((DeleteColumnStatisticsForPartitionRequest) request).withCatalogId(catalogId); + return ((DeleteColumnStatisticsForPartitionRequest) request).toBuilder().catalogId(catalogId).build(); } throw new IllegalArgumentException("Unsupported request: " + request); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueClientUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueClientUtil.java index 03bddd8d8b84..d0628fd565e8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueClientUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueClientUtil.java @@ -13,57 +13,99 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.metrics.RequestMetricCollector; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; import com.google.common.collect.ImmutableList; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.metrics.MetricPublisher; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.GlueAsyncClientBuilder; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.GlueClientBuilder; +import java.net.URI; import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.hdfs.s3.AwsCurrentRegionHolder.getCurrentRegionFromEC2Metadata; public final class GlueClientUtil { private GlueClientUtil() {} - public static AWSGlueAsync createAsyncGlueClient( + public static GlueAsyncClient createAsyncGlueClient( GlueHiveMetastoreConfig config, - AWSCredentialsProvider credentialsProvider, - Optional requestHandler, - RequestMetricCollector metricsCollector) + AwsCredentialsProvider credentialsProvider, + Optional requestHandler, + MetricPublisher metricPublisher) { - ClientConfiguration clientConfig = new ClientConfiguration() - .withMaxConnections(config.getMaxGlueConnections()) - .withMaxErrorRetry(config.getMaxGlueErrorRetries()); - AWSGlueAsyncClientBuilder asyncGlueClientBuilder = AWSGlueAsyncClientBuilder.standard() - .withMetricsCollector(metricsCollector) - .withClientConfiguration(clientConfig); - - ImmutableList.Builder requestHandlers = ImmutableList.builder(); - requestHandler.ifPresent(requestHandlers::add); - config.getCatalogId().ifPresent(catalogId -> requestHandlers.add(new GlueCatalogIdRequestHandler(catalogId))); - asyncGlueClientBuilder.setRequestHandlers(requestHandlers.build().toArray(RequestHandler2[]::new)); + NettyNioAsyncHttpClient.Builder nettyBuilder = NettyNioAsyncHttpClient.builder() + .maxConcurrency(config.getMaxGlueConnections()); + ClientOverrideConfiguration clientOverrideConfiguration = + createClientOverrideConfiguration(config, requestHandler, metricPublisher); + GlueAsyncClientBuilder glueAsyncClientBuilder = GlueAsyncClient.builder() + .httpClient(nettyBuilder.build()) + .overrideConfiguration(clientOverrideConfiguration); if (config.getGlueEndpointUrl().isPresent()) { checkArgument(config.getGlueRegion().isPresent(), "Glue region must be set when Glue endpoint URL is set"); - asyncGlueClientBuilder.setEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( - config.getGlueEndpointUrl().get(), - config.getGlueRegion().get())); + glueAsyncClientBuilder + .endpointOverride(URI.create(config.getGlueEndpointUrl().get())) + .region(Region.of(config.getGlueRegion().get())); } else if (config.getGlueRegion().isPresent()) { - asyncGlueClientBuilder.setRegion(config.getGlueRegion().get()); + glueAsyncClientBuilder.region(Region.of(config.getGlueRegion().get())); } - else if (config.getPinGlueClientToCurrentRegion()) { - asyncGlueClientBuilder.setRegion(getCurrentRegionFromEC2Metadata().getName()); + glueAsyncClientBuilder.credentialsProvider(credentialsProvider); + + return glueAsyncClientBuilder.build(); + } + + public static GlueClient createSyncGlueClient( + GlueHiveMetastoreConfig config, + AwsCredentialsProvider credentialsProvider, + Optional requestHandler, + MetricPublisher metricPublisher) + { + ApacheHttpClient.Builder apacheHttpClientbuilder = ApacheHttpClient.builder() + .maxConnections(config.getMaxGlueConnections()); + + ClientOverrideConfiguration clientOverrideConfiguration = + createClientOverrideConfiguration(config, requestHandler, metricPublisher); + + GlueClientBuilder glueClientBuilder = GlueClient.builder() + .httpClient(apacheHttpClientbuilder.build()) + .overrideConfiguration(clientOverrideConfiguration); + + if (config.getGlueEndpointUrl().isPresent()) { + checkArgument(config.getGlueRegion().isPresent(), "Glue region must be set when Glue endpoint URL is set"); + glueClientBuilder + .endpointOverride(URI.create(config.getGlueEndpointUrl().get())) + .region(Region.of(config.getGlueRegion().get())); + } + else if (config.getGlueRegion().isPresent()) { + glueClientBuilder.region(Region.of(config.getGlueRegion().get())); } + glueClientBuilder.credentialsProvider(credentialsProvider); + return glueClientBuilder.build(); + } - asyncGlueClientBuilder.setCredentials(credentialsProvider); + private static ClientOverrideConfiguration createClientOverrideConfiguration( + GlueHiveMetastoreConfig config, Optional requestHandler, + MetricPublisher metricPublisher) + { + RetryPolicy.Builder retryPolicy = RetryPolicy.builder().numRetries(config.getMaxGlueErrorRetries()); + ClientOverrideConfiguration.Builder clientOverrideConfiguration = ClientOverrideConfiguration.builder() + .addMetricPublisher(metricPublisher) + .retryPolicy(retryPolicy.build()); - return asyncGlueClientBuilder.build(); + ImmutableList.Builder requestHandlers = ImmutableList.builder(); + requestHandler.ifPresent(requestHandlers::add); + config.getCatalogId().ifPresent(catalogId -> requestHandlers.add(new GlueCatalogIdRequestHandler(catalogId))); + clientOverrideConfiguration.executionInterceptors(requestHandlers.build()); + return clientOverrideConfiguration.build(); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueColumnStatisticsProviderFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueColumnStatisticsProviderFactory.java index 10a1d38c7055..0fef153a8a12 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueColumnStatisticsProviderFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueColumnStatisticsProviderFactory.java @@ -13,9 +13,9 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.AWSGlueAsync; +import software.amazon.awssdk.services.glue.GlueAsyncClient; public interface GlueColumnStatisticsProviderFactory { - GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(AWSGlueAsync glueClient, GlueMetastoreStats stats); + GlueColumnStatisticsProvider createGlueColumnStatisticsProvider(GlueAsyncClient glueClient, GlueMetastoreStats stats); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCredentialsProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCredentialsProvider.java index a6f13e0b6d2a..a282beaeac9e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCredentialsProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueCredentialsProvider.java @@ -13,23 +13,26 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSStaticCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; -import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; import com.google.inject.Inject; import com.google.inject.Provider; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +import java.net.URI; -import static io.trino.hdfs.s3.AwsCurrentRegionHolder.getCurrentRegionFromEC2Metadata; import static java.lang.String.format; public class GlueCredentialsProvider - implements Provider + implements Provider { - private final AWSCredentialsProvider credentialsProvider; + private final AwsCredentialsProvider credentialsProvider; @Inject public GlueCredentialsProvider(GlueHiveMetastoreConfig config) @@ -38,33 +41,34 @@ public GlueCredentialsProvider(GlueHiveMetastoreConfig config) this.credentialsProvider = getCustomAWSCredentialsProvider(config.getAwsCredentialsProvider().get()); } else { - AWSCredentialsProvider provider; + AwsCredentialsProvider provider; if (config.getAwsAccessKey().isPresent() && config.getAwsSecretKey().isPresent()) { - provider = new AWSStaticCredentialsProvider( - new BasicAWSCredentials(config.getAwsAccessKey().get(), config.getAwsSecretKey().get())); + provider = StaticCredentialsProvider.create( + AwsBasicCredentials.create(config.getAwsAccessKey().get(), config.getAwsSecretKey().get())); } else { - provider = DefaultAWSCredentialsProviderChain.getInstance(); + provider = DefaultCredentialsProvider.create(); } if (config.getIamRole().isPresent()) { - AWSSecurityTokenServiceClientBuilder stsClientBuilder = AWSSecurityTokenServiceClientBuilder - .standard() - .withCredentials(provider); + StsClientBuilder stsClientBuilder = StsClient.builder() + .credentialsProvider(provider); if (config.getGlueStsEndpointUrl().isPresent() && config.getGlueStsRegion().isPresent()) { - stsClientBuilder.setEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(config.getGlueStsEndpointUrl().get(), config.getGlueStsRegion().get())); + stsClientBuilder.endpointOverride(URI.create(config.getGlueStsEndpointUrl().get())) + .region(Region.of(config.getGlueStsRegion().get())); } else if (config.getGlueStsRegion().isPresent()) { - stsClientBuilder.setRegion(config.getGlueStsRegion().get()); - } - else if (config.getPinGlueClientToCurrentRegion()) { - stsClientBuilder.setRegion(getCurrentRegionFromEC2Metadata().getName()); + stsClientBuilder.region(Region.of(config.getGlueStsRegion().get())); } - provider = new STSAssumeRoleSessionCredentialsProvider - .Builder(config.getIamRole().get(), "trino-session") - .withExternalId(config.getExternalId().orElse(null)) - .withStsClient(stsClientBuilder.build()) + provider = StsAssumeRoleCredentialsProvider.builder() + .refreshRequest(() -> AssumeRoleRequest + .builder() + .roleArn(config.getIamRole().get()) + .roleSessionName("trino-session") + .externalId(config.getExternalId().orElse(null)) + .build()) + .stsClient(stsClientBuilder.build()) .build(); } this.credentialsProvider = provider; @@ -72,19 +76,19 @@ else if (config.getPinGlueClientToCurrentRegion()) { } @Override - public AWSCredentialsProvider get() + public AwsCredentialsProvider get() { return credentialsProvider; } - private static AWSCredentialsProvider getCustomAWSCredentialsProvider(String providerClass) + private static AwsCredentialsProvider getCustomAWSCredentialsProvider(String providerClass) { try { Object instance = Class.forName(providerClass).getConstructor().newInstance(); - if (!(instance instanceof AWSCredentialsProvider)) { + if (!(instance instanceof AwsCredentialsProvider)) { throw new RuntimeException("Invalid credentials provider class: " + instance.getClass().getName()); } - return (AWSCredentialsProvider) instance; + return (AwsCredentialsProvider) instance; } catch (ReflectiveOperationException e) { throw new RuntimeException(format("Error creating an instance of %s", providerClass), e); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java index da3744caa9e1..e5a1490335fc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -13,49 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; -import com.amazonaws.handlers.AsyncHandler; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.AccessDeniedException; -import com.amazonaws.services.glue.model.AlreadyExistsException; -import com.amazonaws.services.glue.model.BatchCreatePartitionRequest; -import com.amazonaws.services.glue.model.BatchCreatePartitionResult; -import com.amazonaws.services.glue.model.BatchGetPartitionRequest; -import com.amazonaws.services.glue.model.BatchGetPartitionResult; -import com.amazonaws.services.glue.model.BatchUpdatePartitionRequest; -import com.amazonaws.services.glue.model.BatchUpdatePartitionRequestEntry; -import com.amazonaws.services.glue.model.BatchUpdatePartitionResult; -import com.amazonaws.services.glue.model.ConcurrentModificationException; -import com.amazonaws.services.glue.model.CreateDatabaseRequest; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.DatabaseInput; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.DeletePartitionRequest; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.ErrorDetail; -import com.amazonaws.services.glue.model.GetDatabaseRequest; -import com.amazonaws.services.glue.model.GetDatabaseResult; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetDatabasesResult; -import com.amazonaws.services.glue.model.GetPartitionRequest; -import com.amazonaws.services.glue.model.GetPartitionResult; -import com.amazonaws.services.glue.model.GetPartitionsRequest; -import com.amazonaws.services.glue.model.GetPartitionsResult; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.GetTableResult; -import com.amazonaws.services.glue.model.GetTablesRequest; -import com.amazonaws.services.glue.model.GetTablesResult; -import com.amazonaws.services.glue.model.PartitionError; -import com.amazonaws.services.glue.model.PartitionInput; -import com.amazonaws.services.glue.model.PartitionValueList; -import com.amazonaws.services.glue.model.Segment; -import com.amazonaws.services.glue.model.TableInput; -import com.amazonaws.services.glue.model.UpdateDatabaseRequest; -import com.amazonaws.services.glue.model.UpdatePartitionRequest; -import com.amazonaws.services.glue.model.UpdateTableRequest; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; @@ -113,6 +70,46 @@ import org.apache.hadoop.fs.Path; import org.weakref.jmx.Flatten; import org.weakref.jmx.Managed; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.AccessDeniedException; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.BatchCreatePartitionRequest; +import software.amazon.awssdk.services.glue.model.BatchCreatePartitionResponse; +import software.amazon.awssdk.services.glue.model.BatchGetPartitionRequest; +import software.amazon.awssdk.services.glue.model.BatchGetPartitionResponse; +import software.amazon.awssdk.services.glue.model.BatchUpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.BatchUpdatePartitionRequestEntry; +import software.amazon.awssdk.services.glue.model.BatchUpdatePartitionResponse; +import software.amazon.awssdk.services.glue.model.ConcurrentModificationException; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeletePartitionRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.ErrorDetail; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabaseResponse; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionResponse; +import software.amazon.awssdk.services.glue.model.GetPartitionsRequest; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTableResponse; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.GlueResponse; +import software.amazon.awssdk.services.glue.model.PartitionError; +import software.amazon.awssdk.services.glue.model.PartitionInput; +import software.amazon.awssdk.services.glue.model.PartitionValueList; +import software.amazon.awssdk.services.glue.model.Segment; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.paginators.GetPartitionsPublisher; import java.time.Duration; import java.util.AbstractMap.SimpleEntry; @@ -124,6 +121,7 @@ import java.util.Optional; import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; @@ -135,7 +133,6 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Comparators.lexicographical; -import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; @@ -145,7 +142,8 @@ import static io.trino.plugin.hive.metastore.MetastoreUtil.makePartitionName; import static io.trino.plugin.hive.metastore.MetastoreUtil.toPartitionName; import static io.trino.plugin.hive.metastore.MetastoreUtil.verifyCanDropColumn; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncPaginatedRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.plugin.hive.metastore.glue.GlueClientUtil.createAsyncGlueClient; import static io.trino.plugin.hive.metastore.glue.converter.GlueInputConverter.convertPartition; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableParameters; @@ -177,7 +175,7 @@ public class GlueHiveMetastore private static final int BATCH_UPDATE_PARTITION_MAX_PAGE_SIZE = 100; private static final int AWS_GLUE_GET_PARTITIONS_MAX_RESULTS = 1000; private static final Comparator> PARTITION_VALUE_COMPARATOR = lexicographical(String.CASE_INSENSITIVE_ORDER); - private static final Predicate VIEWS_FILTER = table -> VIRTUAL_VIEW.name().equals(getTableTypeNullable(table)); + private static final Predicate VIEWS_FILTER = table -> VIRTUAL_VIEW.name().equals(getTableTypeNullable(table)); private static final RetryPolicy CONCURRENT_MODIFICATION_EXCEPTION_RETRY_POLICY = RetryPolicy.builder() .handleIf(throwable -> Throwables.getRootCause(throwable) instanceof ConcurrentModificationException) .withDelay(Duration.ofMillis(100)) @@ -186,14 +184,14 @@ public class GlueHiveMetastore private final HdfsEnvironment hdfsEnvironment; private final HdfsContext hdfsContext; - private final AWSGlueAsync glueClient; + private final GlueAsyncClient glueClient; private final Optional defaultDir; private final int partitionSegments; private final Executor partitionsReadExecutor; private final GlueMetastoreStats stats; private final GlueColumnStatisticsProvider columnStatisticsProvider; private final boolean assumeCanonicalPartitionKeys; - private final Predicate tableFilter; + private final Predicate tableFilter; @Inject public GlueHiveMetastore( @@ -201,9 +199,9 @@ public GlueHiveMetastore( GlueHiveMetastoreConfig glueConfig, @ForGlueHiveMetastore Executor partitionsReadExecutor, GlueColumnStatisticsProviderFactory columnStatisticsProviderFactory, - AWSGlueAsync glueClient, + GlueAsyncClient glueClient, @ForGlueHiveMetastore GlueMetastoreStats stats, - @ForGlueHiveMetastore Predicate tableFilter) + @ForGlueHiveMetastore Predicate tableFilter) { this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.hdfsContext = new HdfsContext(ConnectorIdentity.ofUser(DEFAULT_METASTORE_USER)); @@ -231,7 +229,7 @@ public static GlueHiveMetastore createTestingGlueHiveMetastore(java.nio.file.Pat glueConfig, directExecutor(), new DefaultGlueColumnStatisticsProviderFactory(directExecutor(), directExecutor()), - createAsyncGlueClient(glueConfig, DefaultAWSCredentialsProviderChain.getInstance(), Optional.empty(), stats.newRequestMetricsCollector()), + createAsyncGlueClient(glueConfig, DefaultCredentialsProvider.create(), Optional.empty(), stats.newRequestMetricsPublisher()), stats, table -> true); } @@ -247,14 +245,15 @@ public GlueMetastoreStats getStats() public Optional getDatabase(String databaseName) { try { - GetDatabaseResult result = stats.getGetDatabase().call(() -> - glueClient.getDatabase(new GetDatabaseRequest().withName(databaseName))); - return Optional.of(GlueToTrinoConverter.convertDatabase(result.getDatabase())); + GetDatabaseResponse result = awsSyncRequest(glueClient::getDatabase, + GetDatabaseRequest.builder().name(databaseName).build(), + stats.getGetDatabase()); + return Optional.of(GlueToTrinoConverter.convertDatabase(result.database())); } catch (EntityNotFoundException e) { return Optional.empty(); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -263,19 +262,14 @@ public Optional getDatabase(String databaseName) public List getAllDatabases() { try { - List databaseNames = getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - stats.getGetDatabases()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .map(com.amazonaws.services.glue.model.Database::getName) - .collect(toImmutableList()); - return databaseNames; - } - catch (AmazonServiceException e) { + ImmutableList.Builder databaseNames = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getDatabasesPaginator(GetDatabasesRequest.builder().build()), + getDatabasesResponse -> getDatabasesResponse.databaseList().forEach(database -> databaseNames.add(database.name())), + stats.getGetDatabases()); + + return databaseNames.build(); + } + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -284,16 +278,15 @@ public List getAllDatabases() public Optional getTable(String databaseName, String tableName) { try { - GetTableResult result = stats.getGetTable().call(() -> - glueClient.getTable(new GetTableRequest() - .withDatabaseName(databaseName) - .withName(tableName))); - return Optional.of(GlueToTrinoConverter.convertTable(result.getTable(), databaseName)); + GetTableResponse result = awsSyncRequest(glueClient::getTable, + GetTableRequest.builder().databaseName(databaseName).name(tableName).build(), + stats.getGetTable()); + return Optional.of(GlueToTrinoConverter.convertTable(result.table(), databaseName)); } catch (EntityNotFoundException e) { return Optional.empty(); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -352,17 +345,17 @@ public void updateTableStatistics(String databaseName, String tableName, AcidTra try { TableInput tableInput = GlueInputConverter.convertTable(table); final Map statisticsParameters = updateStatisticsParameters(table.getParameters(), updatedStatistics.getBasicStatistics()); - tableInput.setParameters(statisticsParameters); table = Table.builder(table).setParameters(statisticsParameters).build(); - stats.getUpdateTable().call(() -> glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(databaseName) - .withTableInput(tableInput))); + awsSyncRequest(glueClient::updateTable, UpdateTableRequest.builder() + .databaseName(databaseName) + .tableInput(tableInput.toBuilder().parameters(statisticsParameters).build()) + .build(), stats.getUpdateTable()); columnStatisticsProvider.updateTableColumnStatistics(table, updatedStatistics.getColumnStatistics()); } catch (EntityNotFoundException e) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -397,23 +390,31 @@ private void updatePartitionStatisticsBatch(Table table, Map updatedColumnStatistics = updatedStatistics.getColumnStatistics(); PartitionInput partitionInput = GlueInputConverter.convertPartition(partition); - partitionInput.setParameters(partition.getParameters()); - partitionUpdateRequests.add(new BatchUpdatePartitionRequestEntry() - .withPartitionValueList(partition.getValues()) - .withPartitionInput(partitionInput)); + partitionUpdateRequests.add(BatchUpdatePartitionRequestEntry.builder() + .partitionValueList(partition.getValues()) + .partitionInput(partitionInput.toBuilder().parameters(partition.getParameters()).build()) + .build()); columnStatisticsUpdates.add(new GlueColumnStatisticsProvider.PartitionStatisticsUpdate(partition, updatedColumnStatistics)); }); List> partitionUpdateRequestsPartitioned = Lists.partition(partitionUpdateRequests.build(), BATCH_UPDATE_PARTITION_MAX_PAGE_SIZE); - List> partitionUpdateRequestsFutures = new ArrayList<>(); + List> partitionUpdateRequestsFutures = new ArrayList<>(); partitionUpdateRequestsPartitioned.forEach(partitionUpdateRequestsPartition -> { // Update basic statistics - partitionUpdateRequestsFutures.add(glueClient.batchUpdatePartitionAsync(new BatchUpdatePartitionRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withEntries(partitionUpdateRequestsPartition), - new StatsRecordingAsyncHandler<>(stats.getBatchUpdatePartition()))); + StatsRecordingAsyncHandler statsRecorder = new StatsRecordingAsyncHandler(stats.getBatchUpdatePartition()); + partitionUpdateRequestsFutures.add(glueClient.batchUpdatePartition(BatchUpdatePartitionRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .entries(partitionUpdateRequestsPartition).build()) + .whenCompleteAsync((result, error) -> { + if (result != null) { + statsRecorder.onSuccess(result); + } + else if (error != null) { + statsRecorder.onError(error); + } + })); }); try { @@ -422,7 +423,7 @@ private void updatePartitionStatisticsBatch(Table table, Map getAllTables(String databaseName) { try { - List tableNames = getPaginatedResults( - glueClient::getTables, - new GetTablesRequest() - .withDatabaseName(databaseName), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - stats.getGetTables()) - .map(GetTablesResult::getTableList) - .flatMap(List::stream) - .filter(tableFilter) - .map(com.amazonaws.services.glue.model.Table::getName) - .collect(toImmutableList()); - return tableNames; + ImmutableList.Builder tableNamesBuilder = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getTablesPaginator(GetTablesRequest.builder().databaseName(databaseName).build()), + tables -> { + tables.tableList().stream() + .filter(tableFilter) + .map(software.amazon.awssdk.services.glue.model.Table::name) + .forEach(tableNamesBuilder::add); + }, + stats.getGetTables()); + return tableNamesBuilder.build(); } catch (EntityNotFoundException | AccessDeniedException e) { // database does not exist or permission denied return ImmutableList.of(); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -478,28 +476,25 @@ public Optional> getAllViews() return Optional.empty(); } - private List getAllViews(String databaseName, Predicate additionalFilter) + private List getAllViews(String databaseName, Predicate additionalFilter) { try { - List views = getPaginatedResults( - glueClient::getTables, - new GetTablesRequest() - .withDatabaseName(databaseName), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - stats.getGetTables()) - .map(GetTablesResult::getTableList) - .flatMap(List::stream) - .filter(VIEWS_FILTER.and(additionalFilter)) - .map(com.amazonaws.services.glue.model.Table::getName) - .collect(toImmutableList()); - return views; + ImmutableList.Builder viewsBuilder = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getTablesPaginator(GetTablesRequest.builder().databaseName(databaseName).build()), + tables -> { + tables.tableList().stream() + .filter(VIEWS_FILTER.and(additionalFilter)) + .map(software.amazon.awssdk.services.glue.model.Table::name) + .forEach(viewsBuilder::add); + }, + stats.getGetTables()); + return viewsBuilder.build(); } catch (EntityNotFoundException | AccessDeniedException e) { // database does not exist or permission denied return ImmutableList.of(); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -516,13 +511,14 @@ public void createDatabase(Database database) try { DatabaseInput databaseInput = GlueInputConverter.convertDatabase(database); - stats.getCreateDatabase().call(() -> - glueClient.createDatabase(new CreateDatabaseRequest().withDatabaseInput(databaseInput))); + awsSyncRequest(glueClient::createDatabase, CreateDatabaseRequest.builder() + .databaseInput(databaseInput) + .build(), stats.getCreateDatabase()); } catch (AlreadyExistsException e) { throw new SchemaAlreadyExistsException(database.getDatabaseName()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } @@ -543,13 +539,14 @@ public void dropDatabase(String databaseName, boolean deleteData) } try { - stats.getDeleteDatabase().call(() -> - glueClient.deleteDatabase(new DeleteDatabaseRequest().withName(databaseName))); + awsSyncRequest(glueClient::deleteDatabase, DeleteDatabaseRequest.builder() + .name(databaseName) + .build(), stats.getDeleteDatabase()); } catch (EntityNotFoundException e) { throw new SchemaNotFoundException(databaseName); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } @@ -563,13 +560,13 @@ public void renameDatabase(String databaseName, String newDatabaseName) { try { Database database = getDatabase(databaseName).orElseThrow(() -> new SchemaNotFoundException(databaseName)); - DatabaseInput renamedDatabase = GlueInputConverter.convertDatabase(database).withName(newDatabaseName); - stats.getUpdateDatabase().call(() -> - glueClient.updateDatabase(new UpdateDatabaseRequest() - .withName(databaseName) - .withDatabaseInput(renamedDatabase))); + DatabaseInput renamedDatabase = GlueInputConverter.convertDatabase(database); + awsSyncRequest(glueClient::updateDatabase, UpdateDatabaseRequest.builder() + .name(databaseName) + .databaseInput(renamedDatabase.toBuilder().name(newDatabaseName).build()) + .build(), stats.getUpdateDatabase()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -585,10 +582,10 @@ public void createTable(Table table, PrincipalPrivileges principalPrivileges) { try { TableInput input = GlueInputConverter.convertTable(table); - stats.getCreateTable().call(() -> - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableInput(input))); + awsSyncRequest(glueClient::createTable, CreateTableRequest.builder() + .databaseName(table.getDatabaseName()) + .tableInput(input) + .build(), stats.getCreateTable()); } catch (AlreadyExistsException e) { throw new TableAlreadyExistsException(new SchemaTableName(table.getDatabaseName(), table.getTableName())); @@ -596,7 +593,7 @@ public void createTable(Table table, PrincipalPrivileges principalPrivileges) catch (EntityNotFoundException e) { throw new SchemaNotFoundException(table.getDatabaseName()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -605,15 +602,15 @@ public void createTable(Table table, PrincipalPrivileges principalPrivileges) public void dropTable(String databaseName, String tableName, boolean deleteData) { Table table = getExistingTable(databaseName, tableName); - DeleteTableRequest deleteTableRequest = new DeleteTableRequest() - .withDatabaseName(databaseName) - .withName(tableName); + try { Failsafe.with(CONCURRENT_MODIFICATION_EXCEPTION_RETRY_POLICY) - .run(() -> stats.getDeleteTable().call(() -> - glueClient.deleteTable(deleteTableRequest))); + .run(() -> awsSyncRequest(glueClient::deleteTable, DeleteTableRequest.builder() + .databaseName(databaseName) + .name(tableName) + .build(), stats.getDeleteTable())); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } @@ -648,15 +645,15 @@ public void replaceTable(String databaseName, String tableName, Table newTable, } try { TableInput newTableInput = GlueInputConverter.convertTable(newTable); - stats.getUpdateTable().call(() -> - glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(databaseName) - .withTableInput(newTableInput))); + awsSyncRequest(glueClient::updateTable, UpdateTableRequest.builder() + .databaseName(databaseName) + .tableInput(newTableInput) + .build(), stats.getUpdateTable()); } catch (EntityNotFoundException e) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -666,14 +663,15 @@ public void renameTable(String databaseName, String tableName, String newDatabas { boolean newTableCreated = false; try { - GetTableRequest getTableRequest = new GetTableRequest().withDatabaseName(databaseName) - .withName(tableName); - GetTableResult glueTable = glueClient.getTable(getTableRequest); - TableInput tableInput = convertGlueTableToTableInput(glueTable.getTable(), newTableName); - CreateTableRequest createTableRequest = new CreateTableRequest() - .withDatabaseName(newDatabaseName) - .withTableInput(tableInput); - stats.getCreateTable().call(() -> glueClient.createTable(createTableRequest)); + GetTableRequest getTableRequest = GetTableRequest.builder() + .databaseName(databaseName).name(tableName).build(); + GetTableResponse glueTable = awsSyncRequest(glueClient::getTable, getTableRequest, stats.getGetTable()); + TableInput tableInput = convertGlueTableToTableInput(glueTable.table(), newTableName); + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(newDatabaseName) + .tableInput(tableInput) + .build(); + awsSyncRequest(glueClient::createTable, createTableRequest, stats.getCreateTable()); newTableCreated = true; dropTable(databaseName, tableName, false); } @@ -692,22 +690,22 @@ public void renameTable(String databaseName, String tableName, String newDatabas } } - private TableInput convertGlueTableToTableInput(com.amazonaws.services.glue.model.Table glueTable, String newTableName) + private TableInput convertGlueTableToTableInput(software.amazon.awssdk.services.glue.model.Table glueTable, String newTableName) { - return new TableInput() - .withName(newTableName) - .withDescription(glueTable.getDescription()) - .withOwner(glueTable.getOwner()) - .withLastAccessTime(glueTable.getLastAccessTime()) - .withLastAnalyzedTime(glueTable.getLastAnalyzedTime()) - .withRetention(glueTable.getRetention()) - .withStorageDescriptor(glueTable.getStorageDescriptor()) - .withPartitionKeys(glueTable.getPartitionKeys()) - .withViewOriginalText(glueTable.getViewOriginalText()) - .withViewExpandedText(glueTable.getViewExpandedText()) - .withTableType(getTableTypeNullable(glueTable)) - .withTargetTable(glueTable.getTargetTable()) - .withParameters(getTableParameters(glueTable)); + return TableInput.builder() + .name(newTableName) + .description(glueTable.description()) + .owner(glueTable.owner()) + .lastAccessTime(glueTable.lastAccessTime()) + .lastAnalyzedTime(glueTable.lastAnalyzedTime()) + .retention(glueTable.retention()) + .storageDescriptor(glueTable.storageDescriptor()) + .partitionKeys(glueTable.partitionKeys()) + .viewOriginalText(glueTable.viewOriginalText()) + .viewExpandedText(glueTable.viewExpandedText()) + .tableType(getTableTypeNullable(glueTable)) + .targetTable(glueTable.targetTable()) + .parameters(getTableParameters(glueTable)).build(); } @Override @@ -727,17 +725,16 @@ public void setTableOwner(String databaseName, String tableName, HivePrincipal p try { Table table = getExistingTable(databaseName, tableName); TableInput newTableInput = GlueInputConverter.convertTable(table); - newTableInput.setOwner(principal.getName()); - - stats.getUpdateTable().call(() -> - glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(databaseName) - .withTableInput(newTableInput))); + UpdateTableRequest updateTableRequest = UpdateTableRequest.builder() + .databaseName(databaseName) + .tableInput(newTableInput.toBuilder().owner(principal.getName()).build()) + .build(); + awsSyncRequest(glueClient::updateTable, updateTableRequest, stats.getUpdateTable()); } catch (EntityNotFoundException e) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -808,17 +805,17 @@ public void dropColumn(String databaseName, String tableName, String columnName) public Optional getPartition(Table table, List partitionValues) { try { - GetPartitionResult result = stats.getGetPartition().call(() -> - glueClient.getPartition(new GetPartitionRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withPartitionValues(partitionValues))); - return Optional.of(new GluePartitionConverter(table).apply(result.getPartition())); + GetPartitionRequest getPartitionRequest = GetPartitionRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .partitionValues(partitionValues).build(); + GetPartitionResponse result = awsSyncRequest(glueClient::getPartition, getPartitionRequest, stats.getGetPartition()); + return Optional.of(new GluePartitionConverter(table).apply(result.partition())); } catch (EntityNotFoundException e) { return Optional.empty(); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -850,7 +847,7 @@ private List> getPartitionValues(String databaseName, String tableN List> partitions = new ArrayList<>(); try { for (int i = 0; i < partitionSegments; i++) { - Segment segment = new Segment().withSegmentNumber(i).withTotalSegments(partitionSegments); + Segment segment = Segment.builder().segmentNumber(i).totalSegments(partitionSegments).build(); futures.add(completionService.submit(() -> getPartitionValues(databaseName, tableName, expression, segment))); } for (int i = 0; i < partitionSegments; i++) { @@ -878,27 +875,29 @@ private List> getPartitionValues(String databaseName, String tableN private List> getPartitionValues(String databaseName, String tableName, String expression, @Nullable Segment segment) { try { - // Reuse immutable field instances opportunistically between partitions - return getPaginatedResults( - glueClient::getPartitions, - new GetPartitionsRequest() - .withDatabaseName(databaseName) - .withTableName(tableName) - .withExpression(expression) - .withSegment(segment) - // We are interested in the partition values and excluding column schema - // avoids the problem of a large response. - .withExcludeColumnSchema(true) - .withMaxResults(AWS_GLUE_GET_PARTITIONS_MAX_RESULTS), - GetPartitionsRequest::setNextToken, - GetPartitionsResult::getNextToken, - stats.getGetPartitions()) - .map(GetPartitionsResult::getPartitions) - .flatMap(List::stream) - .map(com.amazonaws.services.glue.model.Partition::getValues) - .collect(toImmutableList()); - } - catch (AmazonServiceException e) { + GetPartitionsRequest partitionsRequest = GetPartitionsRequest.builder() + .databaseName(databaseName) + .tableName(tableName) + .expression(expression) + .segment(segment) + // We are interested in the partition values and excluding column schema + // avoids the problem of a large response. + .excludeColumnSchema(true) + .maxResults(AWS_GLUE_GET_PARTITIONS_MAX_RESULTS).build(); + + ImmutableList.Builder> partitionValuesBuilder = ImmutableList.builder(); + GetPartitionsPublisher partitionsPaginator = glueClient.getPartitionsPaginator(partitionsRequest); + awsSyncPaginatedRequest(partitionsPaginator, + getPartitionsResponse -> { + getPartitionsResponse.partitions().stream() + .map(software.amazon.awssdk.services.glue.model.Partition::values) + .forEach(partitionValuesBuilder::add); + }, + stats.getGetPartitions()); + + return partitionValuesBuilder.build(); + } + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -947,10 +946,10 @@ private Map> getPartitionsByNamesInternal(Table tabl private List batchGetPartition(Table table, List partitionNames) { - List> batchGetPartitionFutures = new ArrayList<>(); + List> batchGetPartitionFutures = new ArrayList<>(); try { List pendingPartitions = partitionNames.stream() - .map(partitionName -> new PartitionValueList().withValues(toPartitionValues(partitionName))) + .map(partitionName -> PartitionValueList.builder().values(toPartitionValues(partitionName)).build()) .collect(toCollection(ArrayList::new)); ImmutableList.Builder resultsBuilder = ImmutableList.builderWithExpectedSize(partitionNames.size()); @@ -960,18 +959,26 @@ private List batchGetPartition(Table table, List partitionNam while (!pendingPartitions.isEmpty()) { for (List partitions : Lists.partition(pendingPartitions, BATCH_GET_PARTITION_MAX_PAGE_SIZE)) { - batchGetPartitionFutures.add(glueClient.batchGetPartitionAsync(new BatchGetPartitionRequest() - .withDatabaseName(table.getDatabaseName()) - .withTableName(table.getTableName()) - .withPartitionsToGet(partitions), - new StatsRecordingAsyncHandler<>(stats.getGetPartitions()))); + StatsRecordingAsyncHandler statsRecorder = new StatsRecordingAsyncHandler(stats.getGetPartitions()); + batchGetPartitionFutures.add(glueClient.batchGetPartition(BatchGetPartitionRequest.builder() + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) + .partitionsToGet(partitions).build()) + .whenCompleteAsync((result, error) -> { + if (result != null) { + statsRecorder.onSuccess(result); + } + else if (error != null) { + statsRecorder.onError(error); + } + })); } pendingPartitions.clear(); - for (Future future : batchGetPartitionFutures) { - BatchGetPartitionResult batchGetPartitionResult = future.get(); - List partitions = batchGetPartitionResult.getPartitions(); - List unprocessedKeys = batchGetPartitionResult.getUnprocessedKeys(); + for (Future futureResponse : batchGetPartitionFutures) { + BatchGetPartitionResponse batchGetPartitionResponse = futureResponse.get(); + List partitions = batchGetPartitionResponse.partitions(); + List unprocessedKeys = batchGetPartitionResponse.unprocessedKeys(); // In the unlikely scenario where batchGetPartition call cannot make progress on retrieving partitions, avoid infinite loop if (partitions.isEmpty()) { @@ -989,7 +996,7 @@ private List batchGetPartition(Table table, List partitionNam return resultsBuilder.build(); } - catch (AmazonServiceException | InterruptedException | ExecutionException e) { + catch (AwsServiceException | InterruptedException | ExecutionException e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } @@ -1006,22 +1013,28 @@ public void addPartitions(String databaseName, String tableName, List { - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); for (List partitionBatch : Lists.partition(partitions, BATCH_CREATE_PARTITION_MAX_PAGE_SIZE)) { List partitionInputs = mappedCopy(partitionBatch, GlueInputConverter::convertPartition); - futures.add(glueClient.batchCreatePartitionAsync( - new BatchCreatePartitionRequest() - .withDatabaseName(databaseName) - .withTableName(tableName) - .withPartitionInputList(partitionInputs), - new StatsRecordingAsyncHandler<>(stats.getBatchCreatePartition()))); + StatsRecordingAsyncHandler statsRecorder = new StatsRecordingAsyncHandler(stats.getBatchCreatePartition()); + futures.add(glueClient.batchCreatePartition(BatchCreatePartitionRequest.builder() + .databaseName(databaseName) + .tableName(tableName) + .partitionInputList(partitionInputs).build()) + .whenCompleteAsync((result, error) -> { + if (result != null) { + statsRecorder.onSuccess(result); + } + else if (error != null) { + statsRecorder.onError(error); + } + })); } - - for (Future future : futures) { + for (Future futureResponse : futures) { try { - BatchCreatePartitionResult result = future.get(); - propagatePartitionErrorToTrinoException(databaseName, tableName, result.getErrors()); + BatchCreatePartitionResponse result = futureResponse.get(); + propagatePartitionErrorToTrinoException(databaseName, tableName, result.errors()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -1039,7 +1052,7 @@ public void addPartitions(String databaseName, String tableName, List partitionErrors) { if (partitionErrors != null && !partitionErrors.isEmpty()) { - ErrorDetail errorDetail = partitionErrors.get(0).getErrorDetail(); - String glueExceptionCode = errorDetail.getErrorCode(); + ErrorDetail errorDetail = partitionErrors.get(0).errorDetail(); + String glueExceptionCode = errorDetail.errorCode(); switch (glueExceptionCode) { case "AlreadyExistsException": - throw new TrinoException(ALREADY_EXISTS, errorDetail.getErrorMessage()); + throw new TrinoException(ALREADY_EXISTS, errorDetail.errorMessage()); case "EntityNotFoundException": - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName), errorDetail.getErrorMessage()); + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName), errorDetail.errorMessage()); default: - throw new TrinoException(HIVE_METASTORE_ERROR, errorDetail.getErrorCode() + ": " + errorDetail.getErrorMessage()); + throw new TrinoException(HIVE_METASTORE_ERROR, errorDetail.errorCode() + ": " + errorDetail.errorMessage()); } } } @@ -1069,13 +1082,13 @@ public void dropPartition(String databaseName, String tableName, List pa .orElseThrow(() -> new PartitionNotFoundException(new SchemaTableName(databaseName, tableName), parts)); try { - stats.getDeletePartition().call(() -> - glueClient.deletePartition(new DeletePartitionRequest() - .withDatabaseName(databaseName) - .withTableName(tableName) - .withPartitionValues(parts))); + awsSyncRequest(glueClient::deletePartition, DeletePartitionRequest.builder() + .databaseName(databaseName) + .tableName(tableName) + .partitionValues(parts) + .build(), stats.getDeletePartition()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } @@ -1090,12 +1103,11 @@ public void alterPartition(String databaseName, String tableName, PartitionWithS { try { PartitionInput newPartition = convertPartition(partition); - stats.getUpdatePartition().call(() -> - glueClient.updatePartition(new UpdatePartitionRequest() - .withDatabaseName(databaseName) - .withTableName(tableName) - .withPartitionInput(newPartition) - .withPartitionValueList(partition.getPartition().getValues()))); + awsSyncRequest(glueClient::updatePartition, UpdatePartitionRequest.builder() + .databaseName(databaseName) + .tableName(tableName) + .partitionInput(newPartition) + .partitionValueList(partition.getPartition().getValues()).build(), stats.getUpdatePartition()); columnStatisticsProvider.updatePartitionStatistics( partition.getPartition(), partition.getStatistics().getColumnStatistics()); @@ -1103,7 +1115,7 @@ public void alterPartition(String databaseName, String tableName, PartitionWithS catch (EntityNotFoundException e) { throw new PartitionNotFoundException(new SchemaTableName(databaseName, tableName), partition.getPartition().getValues()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -1177,8 +1189,7 @@ public void checkSupportsTransactions() throw new TrinoException(NOT_SUPPORTED, "Glue does not support ACID tables"); } - static class StatsRecordingAsyncHandler - implements AsyncHandler + static class StatsRecordingAsyncHandler { private final AwsApiCallStats stats; private final Stopwatch stopwatch; @@ -1189,14 +1200,12 @@ public StatsRecordingAsyncHandler(AwsApiCallStats stats) this.stopwatch = Stopwatch.createStarted(); } - @Override - public void onError(Exception e) + public void onError(Throwable e) { stats.recordCall(stopwatch.elapsed(NANOSECONDS), true); } - @Override - public void onSuccess(AmazonWebServiceRequest request, Object o) + public void onSuccess(GlueResponse response) { stats.recordCall(stopwatch.elapsed(NANOSECONDS), false); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreConfig.java index 9d7873f73087..8989099f2401 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreConfig.java @@ -25,7 +25,7 @@ import static com.google.common.base.Preconditions.checkState; -@DefunctConfig("hive.metastore.glue.use-instance-credentials") +@DefunctConfig({"hive.metastore.glue.use-instance-credentials", "hive.metastore.glue.pin-client-to-current-region"}) public class GlueHiveMetastoreConfig { private Optional glueRegion = Optional.empty(); @@ -33,7 +33,6 @@ public class GlueHiveMetastoreConfig private Optional glueStsRegion = Optional.empty(); private Optional glueStsEndpointUrl = Optional.empty(); private Optional glueProxyApiId = Optional.empty(); - private boolean pinGlueClientToCurrentRegion; private int maxGlueErrorRetries = 10; private int maxGlueConnections = 30; private Optional defaultWarehouseDir = Optional.empty(); @@ -114,19 +113,6 @@ public GlueHiveMetastoreConfig setGlueProxyApiId(String glueProxyApiId) return this; } - public boolean getPinGlueClientToCurrentRegion() - { - return pinGlueClientToCurrentRegion; - } - - @Config("hive.metastore.glue.pin-client-to-current-region") - @ConfigDescription("Should the Glue client be pinned to the current EC2 region") - public GlueHiveMetastoreConfig setPinGlueClientToCurrentRegion(boolean pinGlueClientToCurrentRegion) - { - this.pinGlueClientToCurrentRegion = pinGlueClientToCurrentRegion; - return this; - } - @Min(1) public int getMaxGlueConnections() { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java index 1c855ca4f4fd..8b5a1a7b8118 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java @@ -13,10 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Module; @@ -30,6 +26,11 @@ import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.Table; import java.util.concurrent.Executor; import java.util.function.Predicate; @@ -50,11 +51,11 @@ protected void setup(Binder binder) { GlueHiveMetastoreConfig glueConfig = buildConfigObject(GlueHiveMetastoreConfig.class); glueConfig.getGlueProxyApiId().ifPresent(glueProxyApiId -> binder - .bind(Key.get(RequestHandler2.class, ForGlueHiveMetastore.class)) + .bind(Key.get(ExecutionInterceptor.class, ForGlueHiveMetastore.class)) .toInstance(new ProxyApiRequestHandler(glueProxyApiId))); configBinder(binder).bindConfig(HiveConfig.class); - binder.bind(AWSCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); - newOptionalBinder(binder, Key.get(RequestHandler2.class, ForGlueHiveMetastore.class)); + binder.bind(AwsCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); + newOptionalBinder(binder, Key.get(ExecutionInterceptor.class, ForGlueHiveMetastore.class)); newOptionalBinder(binder, Key.get(new TypeLiteral>() {}, ForGlueHiveMetastore.class)) .setDefault().toProvider(DefaultGlueMetastoreTableFilterProvider.class).in(Scopes.SINGLETON); @@ -68,7 +69,8 @@ protected void setup(Binder binder) // export under the old name, for backwards compatibility binder.bind(GlueHiveMetastoreFactory.class).in(Scopes.SINGLETON); binder.bind(Key.get(GlueMetastoreStats.class, ForGlueHiveMetastore.class)).toInstance(new GlueMetastoreStats()); - binder.bind(AWSGlueAsync.class).toProvider(HiveGlueClientProvider.class).in(Scopes.SINGLETON); + binder.bind(GlueAsyncClient.class).toProvider(HiveGlueAsyncClientProvider.class).in(Scopes.SINGLETON); + binder.bind(GlueClient.class).toProvider(HiveGlueSyncClientProvider.class).in(Scopes.SINGLETON); newExporter(binder).export(GlueHiveMetastoreFactory.class).as(generator -> generator.generatedNameOf(GlueHiveMetastore.class)); binder.bind(Key.get(boolean.class, AllowHiveTableRename.class)).toInstance(false); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreStats.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreStats.java index 7067de2efaa5..ac5842988560 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreStats.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreStats.java @@ -13,12 +13,11 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.metrics.RequestMetricCollector; import io.trino.plugin.hive.aws.AwsApiCallStats; -import io.trino.plugin.hive.aws.AwsSdkClientCoreStats; import org.weakref.jmx.Flatten; import org.weakref.jmx.Managed; import org.weakref.jmx.Nested; +import software.amazon.awssdk.metrics.MetricPublisher; public class GlueMetastoreStats { @@ -48,7 +47,7 @@ public class GlueMetastoreStats private final AwsApiCallStats updateColumnStatisticsForPartition = new AwsApiCallStats(); private final AwsApiCallStats deleteColumnStatisticsForPartition = new AwsApiCallStats(); - private final AwsSdkClientCoreStats clientCoreStats = new AwsSdkClientCoreStats(); + private final GlueSdkClientStats clientCoreStats = new GlueSdkClientStats(); @Managed @Nested @@ -227,13 +226,13 @@ public AwsApiCallStats getDeleteColumnStatisticsForPartition() @Managed @Flatten - public AwsSdkClientCoreStats getClientCoreStats() + public GlueSdkClientStats getClientCoreStats() { return clientCoreStats; } - public RequestMetricCollector newRequestMetricsCollector() + public MetricPublisher newRequestMetricsPublisher() { - return clientCoreStats.newRequestMetricCollector(); + return clientCoreStats.newRequestMetricsPublisher(); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueSdkClientStats.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueSdkClientStats.java new file mode 100644 index 000000000000..6651f6cd0ee4 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueSdkClientStats.java @@ -0,0 +1,181 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.metastore.glue; + +import io.airlift.stats.CounterStat; +import io.airlift.stats.TimeStat; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; +import software.amazon.awssdk.core.internal.metrics.SdkErrorType; +import software.amazon.awssdk.core.metrics.CoreMetric; +import software.amazon.awssdk.http.HttpMetric; +import software.amazon.awssdk.metrics.MetricCollection; +import software.amazon.awssdk.metrics.MetricPublisher; + +import java.time.Duration; +import java.util.concurrent.atomic.AtomicLong; + +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class GlueSdkClientStats +{ + private final CounterStat awsRequestCount = new CounterStat(); + private final CounterStat awsRetryCount = new CounterStat(); + private final CounterStat awsThrottleExceptions = new CounterStat(); + private final TimeStat awsServiceCallDuration = new TimeStat(MILLISECONDS); + private final TimeStat awsApiCallDuration = new TimeStat(MILLISECONDS); + private final TimeStat awsBackoffDelayDuration = new TimeStat(MILLISECONDS); + private final AtomicLong awsHttpClientPoolAvailableCount = new AtomicLong(); + private final AtomicLong awsHttpClientPoolLeasedCount = new AtomicLong(); + private final AtomicLong awsHttpClientPoolPendingCount = new AtomicLong(); + + @Managed + @Nested + public CounterStat getAwsRequestCount() + { + return awsRequestCount; + } + + @Managed + @Nested + public CounterStat getAwsRetryCount() + { + return awsRetryCount; + } + + @Managed + @Nested + public CounterStat getAwsThrottleExceptions() + { + return awsThrottleExceptions; + } + + @Managed + @Nested + public TimeStat getAwsServiceCallDuration() + { + return awsServiceCallDuration; + } + + @Managed + @Nested + public TimeStat getAwsApiCallDuration() + { + return awsApiCallDuration; + } + + @Managed + @Nested + public TimeStat getAwsBackoffDelayDuration() + { + return awsBackoffDelayDuration; + } + + @Managed + public long getAwsHttpClientPoolAvailableCount() + { + return awsHttpClientPoolAvailableCount.get(); + } + + @Managed + public long getAwsHttpClientPoolLeasedCount() + { + return awsHttpClientPoolLeasedCount.get(); + } + + @Managed + public long getAwsHttpClientPoolPendingCount() + { + return awsHttpClientPoolPendingCount.get(); + } + + public GlueSdkClientRequestMetricsPublisher newRequestMetricsPublisher() + { + return new GlueSdkClientRequestMetricsPublisher(this); + } + + public static class GlueSdkClientRequestMetricsPublisher + implements MetricPublisher + { + private final GlueSdkClientStats stats; + + protected GlueSdkClientRequestMetricsPublisher(GlueSdkClientStats stats) + { + this.stats = requireNonNull(stats, "stats is null"); + } + + @Override + public void publish(MetricCollection metricCollection) + { + var requestCount = metricCollection.metricValues(CoreMetric.RETRY_COUNT) + .stream() + .map(i -> i + 1) + .reduce(Integer::sum).orElse(0); + stats.awsRequestCount.update(requestCount); + + var retryCount = metricCollection.metricValues(CoreMetric.RETRY_COUNT) + .stream() + .reduce(Integer::sum).orElse(0); + stats.awsRetryCount.update(retryCount); + + var throttleExceptions = metricCollection + .childrenWithName("ApiCallAttempt") + .flatMap(mc -> mc.metricValues(CoreMetric.ERROR_TYPE).stream()) + .filter(s -> s.equals(SdkErrorType.THROTTLING.toString())) + .count(); + stats.awsThrottleExceptions.update(throttleExceptions); + + var serviceCallDuration = metricCollection + .childrenWithName("ApiCallAttempt") + .flatMap(mc -> mc.metricValues(CoreMetric.SERVICE_CALL_DURATION).stream()) + .reduce(Duration::plus).orElse(Duration.ZERO); + stats.awsServiceCallDuration.add(serviceCallDuration.toMillis(), MILLISECONDS); + + var apiCallDuration = metricCollection + .metricValues(CoreMetric.API_CALL_DURATION) + .stream().reduce(Duration::plus).orElse(Duration.ZERO); + stats.awsApiCallDuration.add(apiCallDuration.toMillis(), MILLISECONDS); + + var backoffDelayDuration = metricCollection + .childrenWithName("ApiCallAttempt") + .flatMap(mc -> mc.metricValues(CoreMetric.BACKOFF_DELAY_DURATION).stream()) + .reduce(Duration::plus).orElse(Duration.ZERO); + stats.awsBackoffDelayDuration.add(backoffDelayDuration.toMillis(), MILLISECONDS); + + var httpClientPoolAvailableCount = metricCollection.childrenWithName("ApiCallAttempt") + .flatMap(attempt -> attempt.childrenWithName("HttpClient")) + .flatMap(httpClient -> httpClient.metricValues(HttpMetric.AVAILABLE_CONCURRENCY).stream()) + .reduce(Integer::max).orElse(0); + stats.awsHttpClientPoolAvailableCount.set(httpClientPoolAvailableCount); + + var httpClientPoolLeasedCount = metricCollection.childrenWithName("ApiCallAttempt") + .flatMap(attempt -> attempt.childrenWithName("HttpClient")) + .flatMap(httpClient -> httpClient.metricValues(HttpMetric.LEASED_CONCURRENCY).stream()) + .reduce(Integer::max).orElse(0); + stats.awsHttpClientPoolLeasedCount.set(httpClientPoolLeasedCount); + + var httpClientPoolPendingCount = metricCollection.childrenWithName("ApiCallAttempt") + .flatMap(attempt -> attempt.childrenWithName("HttpClient")) + .flatMap(httpClient -> httpClient.metricValues(HttpMetric.PENDING_CONCURRENCY_ACQUIRES).stream()) + .reduce(Integer::max).orElse(0); + stats.awsHttpClientPoolPendingCount.set(httpClientPoolPendingCount); + } + + @Override + public void close() + { + } + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueClientProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueAsyncClientProvider.java similarity index 68% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueClientProvider.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueAsyncClientProvider.java index a75b41dea72d..9c3c9516d354 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueClientProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueAsyncClientProvider.java @@ -13,30 +13,30 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.inject.Inject; import com.google.inject.Provider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.GlueAsyncClient; import java.util.Optional; import static io.trino.plugin.hive.metastore.glue.GlueClientUtil.createAsyncGlueClient; import static java.util.Objects.requireNonNull; -public class HiveGlueClientProvider - implements Provider +public class HiveGlueAsyncClientProvider + implements Provider { private final GlueMetastoreStats stats; - private final AWSCredentialsProvider credentialsProvider; + private final AwsCredentialsProvider credentialsProvider; private final GlueHiveMetastoreConfig glueConfig; // TODO do not keep mutable config instance on a field - private final Optional requestHandler; + private final Optional requestHandler; @Inject - public HiveGlueClientProvider( + public HiveGlueAsyncClientProvider( @ForGlueHiveMetastore GlueMetastoreStats stats, - AWSCredentialsProvider credentialsProvider, - @ForGlueHiveMetastore Optional requestHandler, + AwsCredentialsProvider credentialsProvider, + @ForGlueHiveMetastore Optional requestHandler, GlueHiveMetastoreConfig glueConfig) { this.stats = requireNonNull(stats, "stats is null"); @@ -46,8 +46,8 @@ public HiveGlueClientProvider( } @Override - public AWSGlueAsync get() + public GlueAsyncClient get() { - return createAsyncGlueClient(glueConfig, credentialsProvider, requestHandler, stats.newRequestMetricsCollector()); + return createAsyncGlueClient(glueConfig, credentialsProvider, requestHandler, stats.newRequestMetricsPublisher()); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueSyncClientProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueSyncClientProvider.java new file mode 100644 index 000000000000..c9d51812ff68 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/HiveGlueSyncClientProvider.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.metastore.glue; + +import com.google.inject.Inject; +import com.google.inject.Provider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.GlueClient; + +import java.util.Optional; + +import static io.trino.plugin.hive.metastore.glue.GlueClientUtil.createSyncGlueClient; +import static java.util.Objects.requireNonNull; + +public class HiveGlueSyncClientProvider + implements Provider +{ + private final GlueMetastoreStats stats; + private final AwsCredentialsProvider credentialsProvider; + private final GlueHiveMetastoreConfig glueConfig; // TODO do not keep mutable config instance on a field + private final Optional requestHandler; + + @Inject + public HiveGlueSyncClientProvider( + @ForGlueHiveMetastore GlueMetastoreStats stats, + AwsCredentialsProvider credentialsProvider, + @ForGlueHiveMetastore Optional requestHandler, + GlueHiveMetastoreConfig glueConfig) + { + this.stats = requireNonNull(stats, "stats is null"); + this.credentialsProvider = requireNonNull(credentialsProvider, "credentialsProvider is null"); + this.requestHandler = requireNonNull(requestHandler, "requestHandler is null"); + this.glueConfig = glueConfig; + } + + @Override + public GlueClient get() + { + return createSyncGlueClient(glueConfig, credentialsProvider, requestHandler, stats.newRequestMetricsPublisher()); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/ProxyApiRequestHandler.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/ProxyApiRequestHandler.java index 1f395af8b9d0..c260d794217c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/ProxyApiRequestHandler.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/ProxyApiRequestHandler.java @@ -13,14 +13,15 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.Request; -import com.amazonaws.handlers.RequestHandler2; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.http.SdkHttpRequest; import static java.util.Objects.requireNonNull; public class ProxyApiRequestHandler - extends RequestHandler2 + implements ExecutionInterceptor { private final String proxyApiId; @@ -30,19 +31,16 @@ public ProxyApiRequestHandler(String proxyApiId) } @Override - public AmazonWebServiceRequest beforeExecution(AmazonWebServiceRequest request) - { - request.putCustomRequestHeader("x-apigw-api-id", proxyApiId); - return request; - } - - @Override - public void beforeRequest(Request request) + public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) { // AWS Glue SDK will append "X-Amz-Target" header to requests (with "AWSGlue" prefix). // This misleads API Gateway (Glue proxy) that it's not the target of the REST call. Therefore, we // need to pass "X-Amz-Target" value in a special HTTP header that is translated back to "X-Amz-Target" // when API Gateway makes request to AWSGlue. - request.getHeaders().put("X-Trino-Amz-Target-Proxy", request.getHeaders().remove("X-Amz-Target")); + SdkHttpRequest httpRequest = context.httpRequest(); + return httpRequest.toBuilder() + .putHeader("x-apigw-api-id", proxyApiId) + .putHeader("X-Trino-Amz-Target-Proxy", httpRequest.firstMatchingHeader("X-Amz-Target").orElse(null)) + .removeHeader("X-Amz-Target").build(); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueInputConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueInputConverter.java index dd6a818f73c4..90c39b41f1c8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueInputConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueInputConverter.java @@ -13,12 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue.converter; -import com.amazonaws.services.glue.model.DatabaseInput; -import com.amazonaws.services.glue.model.Order; -import com.amazonaws.services.glue.model.PartitionInput; -import com.amazonaws.services.glue.model.SerDeInfo; -import com.amazonaws.services.glue.model.StorageDescriptor; -import com.amazonaws.services.glue.model.TableInput; import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.HiveBucketProperty; import io.trino.plugin.hive.PartitionStatistics; @@ -28,6 +22,12 @@ import io.trino.plugin.hive.metastore.PartitionWithStatistics; import io.trino.plugin.hive.metastore.Storage; import io.trino.plugin.hive.metastore.Table; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.Order; +import software.amazon.awssdk.services.glue.model.PartitionInput; +import software.amazon.awssdk.services.glue.model.SerDeInfo; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.TableInput; import java.util.List; import java.util.Optional; @@ -41,43 +41,44 @@ private GlueInputConverter() {} public static DatabaseInput convertDatabase(Database database) { - DatabaseInput input = new DatabaseInput(); - input.setName(database.getDatabaseName()); - input.setParameters(database.getParameters()); - database.getComment().ifPresent(input::setDescription); - database.getLocation().ifPresent(input::setLocationUri); - return input; + return DatabaseInput.builder() + .name(database.getDatabaseName()) + .parameters(database.getParameters()) + .applyMutation(builder -> database.getComment().ifPresent(builder::description)) + .applyMutation(builder -> database.getLocation().ifPresent(builder::locationUri)) + .build(); } public static TableInput convertTable(Table table) { - TableInput input = new TableInput(); - input.setName(table.getTableName()); - input.setOwner(table.getOwner().orElse(null)); - input.setTableType(table.getTableType()); - input.setStorageDescriptor(convertStorage(table.getStorage(), table.getDataColumns())); - input.setPartitionKeys(table.getPartitionColumns().stream().map(GlueInputConverter::convertColumn).collect(toImmutableList())); - input.setParameters(table.getParameters()); - table.getViewOriginalText().ifPresent(input::setViewOriginalText); - table.getViewExpandedText().ifPresent(input::setViewExpandedText); - return input; + return TableInput.builder() + .name(table.getTableName()) + .owner(table.getOwner().orElse(null)) + .tableType(table.getTableType()) + .storageDescriptor(convertStorage(table.getStorage(), table.getDataColumns())) + .partitionKeys(table.getPartitionColumns().stream().map(GlueInputConverter::convertColumn).collect(toImmutableList())) + .parameters(table.getParameters()) + .applyMutation(builder -> table.getViewOriginalText().ifPresent(builder::viewOriginalText)) + .applyMutation(builder -> table.getViewExpandedText().ifPresent(builder::viewExpandedText)) + .build(); } public static PartitionInput convertPartition(PartitionWithStatistics partitionWithStatistics) { PartitionInput input = convertPartition(partitionWithStatistics.getPartition()); PartitionStatistics statistics = partitionWithStatistics.getStatistics(); - input.setParameters(updateStatisticsParameters(input.getParameters(), statistics.getBasicStatistics())); - return input; + return input.toBuilder() + .parameters(updateStatisticsParameters(input.parameters(), statistics.getBasicStatistics())) + .build(); } public static PartitionInput convertPartition(Partition partition) { - PartitionInput input = new PartitionInput(); - input.setValues(partition.getValues()); - input.setStorageDescriptor(convertStorage(partition.getStorage(), partition.getColumns())); - input.setParameters(partition.getParameters()); - return input; + return PartitionInput.builder() + .values(partition.getValues()) + .storageDescriptor(convertStorage(partition.getStorage(), partition.getColumns())) + .parameters(partition.getParameters()) + .build(); } private static StorageDescriptor convertStorage(Storage storage, List columns) @@ -85,37 +86,39 @@ private static StorageDescriptor convertStorage(Storage storage, List co if (storage.isSkewed()) { throw new IllegalArgumentException("Writing to skewed table/partition is not supported"); } - SerDeInfo serdeInfo = new SerDeInfo() - .withSerializationLibrary(storage.getStorageFormat().getSerDeNullable()) - .withParameters(storage.getSerdeParameters()); + SerDeInfo serdeInfo = SerDeInfo.builder() + .serializationLibrary(storage.getStorageFormat().getSerDeNullable()) + .parameters(storage.getSerdeParameters()) + .build(); - StorageDescriptor sd = new StorageDescriptor(); - sd.setLocation(storage.getLocation()); - sd.setColumns(columns.stream().map(GlueInputConverter::convertColumn).collect(toImmutableList())); - sd.setSerdeInfo(serdeInfo); - sd.setInputFormat(storage.getStorageFormat().getInputFormatNullable()); - sd.setOutputFormat(storage.getStorageFormat().getOutputFormatNullable()); - sd.setParameters(ImmutableMap.of()); + StorageDescriptor.Builder sd = StorageDescriptor.builder() + .location(storage.getLocation()) + .columns(columns.stream().map(GlueInputConverter::convertColumn).collect(toImmutableList())) + .serdeInfo(serdeInfo) + .inputFormat(storage.getStorageFormat().getInputFormatNullable()) + .outputFormat(storage.getStorageFormat().getOutputFormatNullable()) + .parameters(ImmutableMap.of()); Optional bucketProperty = storage.getBucketProperty(); if (bucketProperty.isPresent()) { - sd.setNumberOfBuckets(bucketProperty.get().getBucketCount()); - sd.setBucketColumns(bucketProperty.get().getBucketedBy()); + sd.numberOfBuckets(bucketProperty.get().getBucketCount()); + sd.bucketColumns(bucketProperty.get().getBucketedBy()); if (!bucketProperty.get().getSortedBy().isEmpty()) { - sd.setSortColumns(bucketProperty.get().getSortedBy().stream() - .map(column -> new Order().withColumn(column.getColumnName()).withSortOrder(column.getOrder().getHiveOrder())) + sd.sortColumns(bucketProperty.get().getSortedBy().stream() + .map(column -> Order.builder().column(column.getColumnName()).sortOrder(column.getOrder().getHiveOrder()).build()) .collect(toImmutableList())); } } - return sd; + return sd.build(); } - private static com.amazonaws.services.glue.model.Column convertColumn(Column prestoColumn) + private static software.amazon.awssdk.services.glue.model.Column convertColumn(Column prestoColumn) { - return new com.amazonaws.services.glue.model.Column() - .withName(prestoColumn.getName()) - .withType(prestoColumn.getType().toString()) - .withComment(prestoColumn.getComment().orElse(null)); + return software.amazon.awssdk.services.glue.model.Column.builder() + .name(prestoColumn.getName()) + .type(prestoColumn.getType().toString()) + .comment(prestoColumn.getComment().orElse(null)) + .build(); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueStatConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueStatConverter.java index 1856839f96e8..acac628bab66 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueStatConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueStatConverter.java @@ -13,17 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue.converter; -import com.amazonaws.services.glue.model.BinaryColumnStatisticsData; -import com.amazonaws.services.glue.model.BooleanColumnStatisticsData; -import com.amazonaws.services.glue.model.ColumnStatistics; -import com.amazonaws.services.glue.model.ColumnStatisticsData; -import com.amazonaws.services.glue.model.ColumnStatisticsType; -import com.amazonaws.services.glue.model.DateColumnStatisticsData; -import com.amazonaws.services.glue.model.DecimalColumnStatisticsData; -import com.amazonaws.services.glue.model.DecimalNumber; -import com.amazonaws.services.glue.model.DoubleColumnStatisticsData; -import com.amazonaws.services.glue.model.LongColumnStatisticsData; -import com.amazonaws.services.glue.model.StringColumnStatisticsData; import io.trino.hive.thrift.metastore.Decimal; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.metastore.Column; @@ -33,12 +22,24 @@ import io.trino.plugin.hive.type.PrimitiveTypeInfo; import io.trino.plugin.hive.type.TypeInfo; import io.trino.spi.TrinoException; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.glue.model.BinaryColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.BooleanColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.ColumnStatistics; +import software.amazon.awssdk.services.glue.model.ColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.ColumnStatisticsType; +import software.amazon.awssdk.services.glue.model.DateColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.DecimalColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.DecimalNumber; +import software.amazon.awssdk.services.glue.model.DoubleColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.LongColumnStatisticsData; +import software.amazon.awssdk.services.glue.model.StringColumnStatisticsData; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.time.Instant; import java.time.LocalDate; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.Optional; @@ -93,83 +94,86 @@ public static List toGlueColumnStatistics( private static ColumnStatistics toColumnStatistics(Column column, HiveColumnStatistics statistics, OptionalLong rowCount) { - ColumnStatistics columnStatistics = new ColumnStatistics(); HiveType columnType = column.getType(); - columnStatistics.setColumnName(column.getName()); - columnStatistics.setColumnType(columnType.toString()); ColumnStatisticsData catalogColumnStatisticsData = toGlueColumnStatisticsData(statistics, columnType, rowCount); - columnStatistics.setStatisticsData(catalogColumnStatisticsData); - columnStatistics.setAnalyzedTime(new Date()); + ColumnStatistics columnStatistics = ColumnStatistics.builder() + .columnName(column.getName()) + .columnType(columnType.toString()) + .statisticsData(catalogColumnStatisticsData) + .analyzedTime(Instant.now()) + .build(); return columnStatistics; } public static HiveColumnStatistics fromGlueColumnStatistics(ColumnStatisticsData catalogColumnStatisticsData, OptionalLong rowCount) { - ColumnStatisticsType type = ColumnStatisticsType.fromValue(catalogColumnStatisticsData.getType()); + ColumnStatisticsType type = ColumnStatisticsType.fromValue(catalogColumnStatisticsData.type().toString()); switch (type) { case BINARY: { - BinaryColumnStatisticsData data = catalogColumnStatisticsData.getBinaryColumnStatisticsData(); - OptionalLong max = OptionalLong.of(data.getMaximumLength()); - OptionalDouble avg = OptionalDouble.of(data.getAverageLength()); - OptionalLong nulls = fromMetastoreNullsCount(data.getNumberOfNulls()); + BinaryColumnStatisticsData data = catalogColumnStatisticsData.binaryColumnStatisticsData(); + OptionalLong max = OptionalLong.of(data.maximumLength()); + OptionalDouble avg = OptionalDouble.of(data.averageLength()); + OptionalLong nulls = fromMetastoreNullsCount(data.numberOfNulls()); return createBinaryColumnStatistics( max, getTotalSizeInBytes(avg, rowCount, nulls), nulls); } case BOOLEAN: { - BooleanColumnStatisticsData catalogBooleanData = catalogColumnStatisticsData.getBooleanColumnStatisticsData(); + BooleanColumnStatisticsData catalogBooleanData = catalogColumnStatisticsData.booleanColumnStatisticsData(); return createBooleanColumnStatistics( - OptionalLong.of(catalogBooleanData.getNumberOfTrues()), - OptionalLong.of(catalogBooleanData.getNumberOfFalses()), - fromMetastoreNullsCount(catalogBooleanData.getNumberOfNulls())); + OptionalLong.of(catalogBooleanData.numberOfTrues()), + OptionalLong.of(catalogBooleanData.numberOfFalses()), + fromMetastoreNullsCount(catalogBooleanData.numberOfNulls())); } case DATE: { - DateColumnStatisticsData data = catalogColumnStatisticsData.getDateColumnStatisticsData(); - Optional min = dateToLocalDate(data.getMinimumValue()); - Optional max = dateToLocalDate(data.getMaximumValue()); - OptionalLong nullsCount = fromMetastoreNullsCount(data.getNumberOfNulls()); - OptionalLong distinctValues = OptionalLong.of(data.getNumberOfDistinctValues()); + DateColumnStatisticsData data = catalogColumnStatisticsData.dateColumnStatisticsData(); + Optional min = instantToLocalDate(data.minimumValue()); + Optional max = instantToLocalDate(data.maximumValue()); + OptionalLong nullsCount = fromMetastoreNullsCount(data.numberOfNulls()); + OptionalLong distinctValues = OptionalLong.of(data.numberOfDistinctValues()); return createDateColumnStatistics(min, max, nullsCount, fromMetastoreDistinctValuesCount(distinctValues, nullsCount, rowCount)); } case DECIMAL: { - DecimalColumnStatisticsData data = catalogColumnStatisticsData.getDecimalColumnStatisticsData(); - Optional min = glueDecimalToBigDecimal(data.getMinimumValue()); - Optional max = glueDecimalToBigDecimal(data.getMaximumValue()); - OptionalLong distinctValues = OptionalLong.of(data.getNumberOfDistinctValues()); - OptionalLong nullsCount = fromMetastoreNullsCount(data.getNumberOfNulls()); + DecimalColumnStatisticsData data = catalogColumnStatisticsData.decimalColumnStatisticsData(); + Optional min = glueDecimalToBigDecimal(data.minimumValue()); + Optional max = glueDecimalToBigDecimal(data.maximumValue()); + OptionalLong distinctValues = OptionalLong.of(data.numberOfDistinctValues()); + OptionalLong nullsCount = fromMetastoreNullsCount(data.numberOfNulls()); return createDecimalColumnStatistics(min, max, nullsCount, fromMetastoreDistinctValuesCount(distinctValues, nullsCount, rowCount)); } case DOUBLE: { - DoubleColumnStatisticsData data = catalogColumnStatisticsData.getDoubleColumnStatisticsData(); - OptionalDouble min = OptionalDouble.of(data.getMinimumValue()); - OptionalDouble max = OptionalDouble.of(data.getMaximumValue()); - OptionalLong nulls = fromMetastoreNullsCount(data.getNumberOfNulls()); - OptionalLong distinctValues = OptionalLong.of(data.getNumberOfDistinctValues()); + DoubleColumnStatisticsData data = catalogColumnStatisticsData.doubleColumnStatisticsData(); + OptionalDouble min = OptionalDouble.of(data.minimumValue()); + OptionalDouble max = OptionalDouble.of(data.maximumValue()); + OptionalLong nulls = fromMetastoreNullsCount(data.numberOfNulls()); + OptionalLong distinctValues = OptionalLong.of(data.numberOfDistinctValues()); return createDoubleColumnStatistics(min, max, nulls, fromMetastoreDistinctValuesCount(distinctValues, nulls, rowCount)); } case LONG: { - LongColumnStatisticsData data = catalogColumnStatisticsData.getLongColumnStatisticsData(); - OptionalLong min = OptionalLong.of(data.getMinimumValue()); - OptionalLong max = OptionalLong.of(data.getMaximumValue()); - OptionalLong nullsCount = fromMetastoreNullsCount(data.getNumberOfNulls()); - OptionalLong distinctValues = OptionalLong.of(data.getNumberOfDistinctValues()); + LongColumnStatisticsData data = catalogColumnStatisticsData.longColumnStatisticsData(); + OptionalLong min = OptionalLong.of(data.minimumValue()); + OptionalLong max = OptionalLong.of(data.maximumValue()); + OptionalLong nullsCount = fromMetastoreNullsCount(data.numberOfNulls()); + OptionalLong distinctValues = OptionalLong.of(data.numberOfDistinctValues()); return createIntegerColumnStatistics(min, max, nullsCount, fromMetastoreDistinctValuesCount(distinctValues, nullsCount, rowCount)); } case STRING: { - StringColumnStatisticsData data = catalogColumnStatisticsData.getStringColumnStatisticsData(); - OptionalLong max = OptionalLong.of(data.getMaximumLength()); - OptionalDouble avg = OptionalDouble.of(data.getAverageLength()); - OptionalLong nullsCount = fromMetastoreNullsCount(data.getNumberOfNulls()); - OptionalLong distinctValues = OptionalLong.of(data.getNumberOfDistinctValues()); + StringColumnStatisticsData data = catalogColumnStatisticsData.stringColumnStatisticsData(); + OptionalLong max = OptionalLong.of(data.maximumLength()); + OptionalDouble avg = OptionalDouble.of(data.averageLength()); + OptionalLong nullsCount = fromMetastoreNullsCount(data.numberOfNulls()); + OptionalLong distinctValues = OptionalLong.of(data.numberOfDistinctValues()); return createStringColumnStatistics( max, getTotalSizeInBytes(avg, rowCount, nullsCount), nullsCount, fromMetastoreDistinctValuesCount(distinctValues, nullsCount, rowCount)); } + case UNKNOWN_TO_SDK_VERSION: { + throw new TrinoException(HIVE_INVALID_METADATA, "Invalid column statistics data: " + catalogColumnStatisticsData); + } } - throw new TrinoException(HIVE_INVALID_METADATA, "Invalid column statistics data: " + catalogColumnStatisticsData); } @@ -178,64 +182,64 @@ private static ColumnStatisticsData toGlueColumnStatisticsData(HiveColumnStatist TypeInfo typeInfo = columnType.getTypeInfo(); checkArgument(typeInfo.getCategory() == PRIMITIVE, "Unsupported statistics type: %s", columnType); - ColumnStatisticsData catalogColumnStatisticsData = new ColumnStatisticsData(); + ColumnStatisticsData.Builder catalogColumnStatisticsData = ColumnStatisticsData.builder(); switch (((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) { case BOOLEAN: { - BooleanColumnStatisticsData data = new BooleanColumnStatisticsData(); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); + BooleanColumnStatisticsData.Builder data = BooleanColumnStatisticsData.builder(); + statistics.getNullsCount().ifPresent(data::numberOfNulls); statistics.getBooleanStatistics().ifPresent(booleanStatistics -> { - booleanStatistics.getFalseCount().ifPresent(data::setNumberOfFalses); - booleanStatistics.getTrueCount().ifPresent(data::setNumberOfTrues); + booleanStatistics.getFalseCount().ifPresent(data::numberOfFalses); + booleanStatistics.getTrueCount().ifPresent(data::numberOfTrues); }); - catalogColumnStatisticsData.setType(ColumnStatisticsType.BOOLEAN.toString()); - catalogColumnStatisticsData.setBooleanColumnStatisticsData(data); + catalogColumnStatisticsData.type(ColumnStatisticsType.BOOLEAN.toString()); + catalogColumnStatisticsData.booleanColumnStatisticsData(data.build()); break; } case BINARY: { - BinaryColumnStatisticsData data = new BinaryColumnStatisticsData(); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - data.setMaximumLength(statistics.getMaxValueSizeInBytes().orElse(0)); - data.setAverageLength(getAverageColumnLength(statistics.getTotalSizeInBytes(), rowCount, statistics.getNullsCount()).orElse(0)); - catalogColumnStatisticsData.setType(ColumnStatisticsType.BINARY.toString()); - catalogColumnStatisticsData.setBinaryColumnStatisticsData(data); + BinaryColumnStatisticsData.Builder data = BinaryColumnStatisticsData.builder(); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + data.maximumLength(statistics.getMaxValueSizeInBytes().orElse(0)); + data.averageLength(getAverageColumnLength(statistics.getTotalSizeInBytes(), rowCount, statistics.getNullsCount()).orElse(0)); + catalogColumnStatisticsData.type(ColumnStatisticsType.BINARY.toString()); + catalogColumnStatisticsData.binaryColumnStatisticsData(data.build()); break; } case DATE: { - DateColumnStatisticsData data = new DateColumnStatisticsData(); + DateColumnStatisticsData.Builder data = DateColumnStatisticsData.builder(); statistics.getDateStatistics().ifPresent(dateStatistics -> { - dateStatistics.getMin().ifPresent(value -> data.setMinimumValue(localDateToDate(value))); - dateStatistics.getMax().ifPresent(value -> data.setMaximumValue(localDateToDate(value))); + dateStatistics.getMin().ifPresent(value -> data.minimumValue(localDateToInstant(value))); + dateStatistics.getMax().ifPresent(value -> data.maximumValue(localDateToInstant(value))); }); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::setNumberOfDistinctValues); - catalogColumnStatisticsData.setType(ColumnStatisticsType.DATE.toString()); - catalogColumnStatisticsData.setDateColumnStatisticsData(data); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::numberOfDistinctValues); + catalogColumnStatisticsData.type(ColumnStatisticsType.DATE.toString()); + catalogColumnStatisticsData.dateColumnStatisticsData(data.build()); break; } case DECIMAL: { - DecimalColumnStatisticsData data = new DecimalColumnStatisticsData(); + DecimalColumnStatisticsData.Builder data = DecimalColumnStatisticsData.builder(); statistics.getDecimalStatistics().ifPresent(decimalStatistics -> { - decimalStatistics.getMin().ifPresent(value -> data.setMinimumValue(bigDecimalToGlueDecimal(value))); - decimalStatistics.getMax().ifPresent(value -> data.setMaximumValue(bigDecimalToGlueDecimal(value))); + decimalStatistics.getMin().ifPresent(value -> data.minimumValue(bigDecimalToGlueDecimal(value))); + decimalStatistics.getMax().ifPresent(value -> data.maximumValue(bigDecimalToGlueDecimal(value))); }); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::setNumberOfDistinctValues); - catalogColumnStatisticsData.setType(ColumnStatisticsType.DECIMAL.toString()); - catalogColumnStatisticsData.setDecimalColumnStatisticsData(data); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::numberOfDistinctValues); + catalogColumnStatisticsData.type(ColumnStatisticsType.DECIMAL.toString()); + catalogColumnStatisticsData.decimalColumnStatisticsData(data.build()); break; } case FLOAT: case DOUBLE: { - DoubleColumnStatisticsData data = new DoubleColumnStatisticsData(); + DoubleColumnStatisticsData.Builder data = DoubleColumnStatisticsData.builder(); statistics.getDoubleStatistics().ifPresent(doubleStatistics -> { - doubleStatistics.getMin().ifPresent(data::setMinimumValue); - doubleStatistics.getMax().ifPresent(data::setMaximumValue); + doubleStatistics.getMin().ifPresent(data::minimumValue); + doubleStatistics.getMax().ifPresent(data::maximumValue); }); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::setNumberOfDistinctValues); - catalogColumnStatisticsData.setType(ColumnStatisticsType.DOUBLE.toString()); - catalogColumnStatisticsData.setDoubleColumnStatisticsData(data); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::numberOfDistinctValues); + catalogColumnStatisticsData.type(ColumnStatisticsType.DOUBLE.toString()); + catalogColumnStatisticsData.doubleColumnStatisticsData(data.build()); break; } case BYTE: @@ -243,42 +247,42 @@ private static ColumnStatisticsData toGlueColumnStatisticsData(HiveColumnStatist case INT: case LONG: case TIMESTAMP: { - LongColumnStatisticsData data = new LongColumnStatisticsData(); + LongColumnStatisticsData.Builder data = LongColumnStatisticsData.builder(); statistics.getIntegerStatistics().ifPresent(stats -> { - stats.getMin().ifPresent(data::setMinimumValue); - stats.getMax().ifPresent(data::setMaximumValue); + stats.getMin().ifPresent(data::minimumValue); + stats.getMax().ifPresent(data::maximumValue); }); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::setNumberOfDistinctValues); - catalogColumnStatisticsData.setType(ColumnStatisticsType.LONG.toString()); - catalogColumnStatisticsData.setLongColumnStatisticsData(data); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::numberOfDistinctValues); + catalogColumnStatisticsData.type(ColumnStatisticsType.LONG.toString()); + catalogColumnStatisticsData.longColumnStatisticsData(data.build()); break; } case VARCHAR: case CHAR: case STRING: { - StringColumnStatisticsData data = new StringColumnStatisticsData(); - statistics.getNullsCount().ifPresent(data::setNumberOfNulls); - toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::setNumberOfDistinctValues); - data.setMaximumLength(statistics.getMaxValueSizeInBytes().orElse(0)); - data.setAverageLength(getAverageColumnLength(statistics.getTotalSizeInBytes(), rowCount, statistics.getNullsCount()).orElse(0)); - catalogColumnStatisticsData.setType(ColumnStatisticsType.STRING.toString()); - catalogColumnStatisticsData.setStringColumnStatisticsData(data); + StringColumnStatisticsData.Builder data = StringColumnStatisticsData.builder(); + statistics.getNullsCount().ifPresent(data::numberOfNulls); + toMetastoreDistinctValuesCount(statistics.getDistinctValuesCount(), statistics.getNullsCount()).ifPresent(data::numberOfDistinctValues); + data.maximumLength(statistics.getMaxValueSizeInBytes().orElse(0)); + data.averageLength(getAverageColumnLength(statistics.getTotalSizeInBytes(), rowCount, statistics.getNullsCount()).orElse(0)); + catalogColumnStatisticsData.type(ColumnStatisticsType.STRING.toString()); + catalogColumnStatisticsData.stringColumnStatisticsData(data.build()); break; } default: throw new TrinoException(HIVE_INVALID_METADATA, "Invalid column statistics type: " + ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()); } - return catalogColumnStatisticsData; + return catalogColumnStatisticsData.build(); } private static DecimalNumber bigDecimalToGlueDecimal(BigDecimal decimal) { Decimal hiveDecimal = new Decimal((short) decimal.scale(), ByteBuffer.wrap(decimal.unscaledValue().toByteArray())); - DecimalNumber catalogDecimal = new DecimalNumber(); - catalogDecimal.setUnscaledValue(ByteBuffer.wrap(hiveDecimal.getUnscaled())); - catalogDecimal.setScale((int) hiveDecimal.getScale()); - return catalogDecimal; + return DecimalNumber.builder() + .unscaledValue(SdkBytes.fromByteArray(hiveDecimal.getUnscaled())) + .scale((int) hiveDecimal.getScale()) + .build(); } private static Optional glueDecimalToBigDecimal(DecimalNumber catalogDecimal) @@ -287,23 +291,23 @@ private static Optional glueDecimalToBigDecimal(DecimalNumber catalo return Optional.empty(); } Decimal decimal = new Decimal(); - decimal.setUnscaled(catalogDecimal.getUnscaledValue()); - decimal.setScale(catalogDecimal.getScale().shortValue()); + decimal.setUnscaled(catalogDecimal.unscaledValue().asByteBuffer()); + decimal.setScale(catalogDecimal.scale().shortValue()); return Optional.of(new BigDecimal(new BigInteger(decimal.getUnscaled()), decimal.getScale())); } - private static Optional dateToLocalDate(Date date) + private static Optional instantToLocalDate(Instant date) { if (date == null) { return Optional.empty(); } - long daysSinceEpoch = date.getTime() / MILLIS_PER_DAY; + long daysSinceEpoch = date.toEpochMilli() / MILLIS_PER_DAY; return Optional.of(LocalDate.ofEpochDay(daysSinceEpoch)); } - private static Date localDateToDate(LocalDate date) + private static Instant localDateToInstant(LocalDate date) { long millisecondsSinceEpoch = date.toEpochDay() * MILLIS_PER_DAY; - return new Date(millisecondsSinceEpoch); + return Instant.ofEpochMilli(millisecondsSinceEpoch); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java index dd641772ec7e..c1fae67ae6df 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java @@ -13,8 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue.converter; -import com.amazonaws.services.glue.model.SerDeInfo; -import com.amazonaws.services.glue.model.StorageDescriptor; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.HiveBucketProperty; @@ -35,6 +33,8 @@ import io.trino.spi.security.PrincipalType; import jakarta.annotation.Nullable; import org.gaul.modernizer_maven_annotations.SuppressModernizer; +import software.amazon.awssdk.services.glue.model.SerDeInfo; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; import java.util.List; import java.util.Locale; @@ -63,7 +63,7 @@ public final class GlueToTrinoConverter private GlueToTrinoConverter() {} - public static String getTableType(com.amazonaws.services.glue.model.Table glueTable) + public static String getTableType(software.amazon.awssdk.services.glue.model.Table glueTable) { // Athena treats missing table type as EXTERNAL_TABLE. return firstNonNull(getTableTypeNullable(glueTable), EXTERNAL_TABLE.name()); @@ -71,60 +71,60 @@ public static String getTableType(com.amazonaws.services.glue.model.Table glueTa @Nullable @SuppressModernizer // Usage of `Table.getTableType` is not allowed. Only this method can call that. - public static String getTableTypeNullable(com.amazonaws.services.glue.model.Table glueTable) + public static String getTableTypeNullable(software.amazon.awssdk.services.glue.model.Table glueTable) { - return glueTable.getTableType(); + return glueTable.tableType(); } @SuppressModernizer // Usage of `Table.getParameters` is not allowed. Only this method can call that. - public static Map getTableParameters(com.amazonaws.services.glue.model.Table glueTable) + public static Map getTableParameters(software.amazon.awssdk.services.glue.model.Table glueTable) { - return firstNonNull(glueTable.getParameters(), ImmutableMap.of()); + return firstNonNull(glueTable.parameters(), ImmutableMap.of()); } @SuppressModernizer // Usage of `Partition.getParameters` is not allowed. Only this method can call that. - public static Map getPartitionParameters(com.amazonaws.services.glue.model.Partition gluePartition) + public static Map getPartitionParameters(software.amazon.awssdk.services.glue.model.Partition gluePartition) { - return firstNonNull(gluePartition.getParameters(), ImmutableMap.of()); + return firstNonNull(gluePartition.parameters(), ImmutableMap.of()); } @SuppressModernizer // Usage of `SerDeInfo.getParameters` is not allowed. Only this method can call that. - public static Map getSerDeInfoParameters(com.amazonaws.services.glue.model.SerDeInfo glueSerDeInfo) + public static Map getSerDeInfoParameters(software.amazon.awssdk.services.glue.model.SerDeInfo glueSerDeInfo) { - return firstNonNull(glueSerDeInfo.getParameters(), ImmutableMap.of()); + return firstNonNull(glueSerDeInfo.parameters(), ImmutableMap.of()); } - public static Database convertDatabase(com.amazonaws.services.glue.model.Database glueDb) + public static Database convertDatabase(software.amazon.awssdk.services.glue.model.Database glueDb) { return Database.builder() - .setDatabaseName(glueDb.getName()) + .setDatabaseName(glueDb.name()) // Currently it's not possible to create a Glue database with empty location string "" // (validation error detected: Value '' at 'database.locationUri' failed to satisfy constraint: Member must have length greater than or equal to 1) // However, it has been observed that Glue databases with empty location do exist in the wild. - .setLocation(Optional.ofNullable(emptyToNull(glueDb.getLocationUri()))) - .setComment(Optional.ofNullable(glueDb.getDescription())) - .setParameters(firstNonNull(glueDb.getParameters(), ImmutableMap.of())) + .setLocation(Optional.ofNullable(emptyToNull(glueDb.locationUri()))) + .setComment(Optional.ofNullable(glueDb.description())) + .setParameters(firstNonNull(glueDb.parameters(), ImmutableMap.of())) .setOwnerName(Optional.of(PUBLIC_OWNER)) .setOwnerType(Optional.of(PrincipalType.ROLE)) .build(); } - public static Table convertTable(com.amazonaws.services.glue.model.Table glueTable, String dbName) + public static Table convertTable(software.amazon.awssdk.services.glue.model.Table glueTable, String dbName) { - SchemaTableName table = new SchemaTableName(dbName, glueTable.getName()); + SchemaTableName table = new SchemaTableName(dbName, glueTable.name()); String tableType = getTableType(glueTable); Map tableParameters = ImmutableMap.copyOf(getTableParameters(glueTable)); Table.Builder tableBuilder = Table.builder() .setDatabaseName(table.getSchemaName()) - .setTableName(table.getTableName()) - .setOwner(Optional.ofNullable(glueTable.getOwner())) + .setTableName(glueTable.name()) + .setOwner(Optional.ofNullable(glueTable.owner())) .setTableType(tableType) .setParameters(tableParameters) - .setViewOriginalText(Optional.ofNullable(glueTable.getViewOriginalText())) - .setViewExpandedText(Optional.ofNullable(glueTable.getViewExpandedText())); + .setViewOriginalText(Optional.ofNullable(glueTable.viewOriginalText())) + .setViewExpandedText(Optional.ofNullable(glueTable.viewExpandedText())); - StorageDescriptor sd = glueTable.getStorageDescriptor(); + StorageDescriptor sd = glueTable.storageDescriptor(); if (isIcebergTable(tableParameters) || (sd == null && isDeltaLakeTable(tableParameters)) || @@ -139,9 +139,9 @@ public static Table convertTable(com.amazonaws.services.glue.model.Table glueTab if (sd == null) { throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, "Table StorageDescriptor is null for table '%s' %s".formatted(table, glueTable)); } - tableBuilder.setDataColumns(convertColumns(table, sd.getColumns(), sd.getSerdeInfo().getSerializationLibrary())); - if (glueTable.getPartitionKeys() != null) { - tableBuilder.setPartitionColumns(convertColumns(table, glueTable.getPartitionKeys(), sd.getSerdeInfo().getSerializationLibrary())); + tableBuilder.setDataColumns(convertColumns(table, sd.columns(), sd.serdeInfo().serializationLibrary())); + if (glueTable.partitionKeys() != null) { + tableBuilder.setPartitionColumns(convertColumns(table, glueTable.partitionKeys(), sd.serdeInfo().serializationLibrary())); } else { tableBuilder.setPartitionColumns(ImmutableList.of()); @@ -153,28 +153,28 @@ public static Table convertTable(com.amazonaws.services.glue.model.Table glueTab return tableBuilder.build(); } - private static Column convertColumn(SchemaTableName table, com.amazonaws.services.glue.model.Column glueColumn, String serde) + private static Column convertColumn(SchemaTableName table, software.amazon.awssdk.services.glue.model.Column glueColumn, String serde) { // OpenCSVSerde deserializes columns from csv file into strings, so we set the column type from the metastore // to string to avoid cast exceptions. if (HiveStorageFormat.CSV.getSerde().equals(serde)) { //TODO(https://github.com/trinodb/trino/issues/7240) Add tests - return new Column(glueColumn.getName(), HiveType.HIVE_STRING, Optional.ofNullable(glueColumn.getComment())); + return new Column(glueColumn.name(), HiveType.HIVE_STRING, Optional.ofNullable(glueColumn.comment())); } - return new Column(glueColumn.getName(), convertType(table, glueColumn), Optional.ofNullable(glueColumn.getComment())); + return new Column(glueColumn.name(), convertType(table, glueColumn), Optional.ofNullable(glueColumn.comment())); } - private static HiveType convertType(SchemaTableName table, com.amazonaws.services.glue.model.Column column) + private static HiveType convertType(SchemaTableName table, software.amazon.awssdk.services.glue.model.Column column) { try { - return HiveType.valueOf(column.getType().toLowerCase(Locale.ENGLISH)); + return HiveType.valueOf(column.type().toLowerCase(Locale.ENGLISH)); } catch (IllegalArgumentException e) { - throw new TrinoException(HIVE_INVALID_METADATA, "Glue table '%s' column '%s' has invalid data type: %s".formatted(table, column.getName(), column.getType())); + throw new TrinoException(HIVE_INVALID_METADATA, "Glue table '%s' column '%s' has invalid data type: %s".formatted(table, column.name(), column.type())); } } - private static List convertColumns(SchemaTableName table, List glueColumns, String serde) + private static List convertColumns(SchemaTableName table, List glueColumns, String serde) { return mappedCopy(glueColumns, glueColumn -> convertColumn(table, glueColumn, serde)); } @@ -190,9 +190,9 @@ private static boolean isNullOrEmpty(List list) } public static final class GluePartitionConverter - implements Function + implements Function { - private final Function, List> columnsConverter; + private final Function, List> columnsConverter; private final Function, Map> parametersConverter = parametersConverter(); private final StorageConverter storageConverter = new StorageConverter(); private final String databaseName; @@ -212,22 +212,22 @@ public GluePartitionConverter(Table table) } @Override - public Partition apply(com.amazonaws.services.glue.model.Partition gluePartition) + public Partition apply(software.amazon.awssdk.services.glue.model.Partition gluePartition) { - requireNonNull(gluePartition.getStorageDescriptor(), "Partition StorageDescriptor is null"); - StorageDescriptor sd = gluePartition.getStorageDescriptor(); + requireNonNull(gluePartition.storageDescriptor(), "Partition StorageDescriptor is null"); + StorageDescriptor sd = gluePartition.storageDescriptor(); - if (!databaseName.equals(gluePartition.getDatabaseName())) { - throw new IllegalArgumentException(format("Unexpected databaseName, expected: %s, but found: %s", databaseName, gluePartition.getDatabaseName())); + if (!databaseName.equals(gluePartition.databaseName())) { + throw new IllegalArgumentException(format("Unexpected databaseName, expected: %s, but found: %s", databaseName, gluePartition.databaseName())); } - if (!tableName.equals(gluePartition.getTableName())) { - throw new IllegalArgumentException(format("Unexpected tableName, expected: %s, but found: %s", tableName, gluePartition.getTableName())); + if (!tableName.equals(gluePartition.tableName())) { + throw new IllegalArgumentException(format("Unexpected tableName, expected: %s, but found: %s", tableName, gluePartition.tableName())); } Partition.Builder partitionBuilder = Partition.builder() .setDatabaseName(databaseName) .setTableName(tableName) - .setValues(gluePartition.getValues()) // No memoization benefit - .setColumns(columnsConverter.apply(sd.getColumns())) + .setValues(gluePartition.values()) // No memoization benefit + .setColumns(columnsConverter.apply(sd.columns())) .setParameters(parametersConverter.apply(getPartitionParameters(gluePartition))); storageConverter.setStorageBuilder(sd, partitionBuilder.getStorageBuilder(), tableParameters); @@ -239,44 +239,44 @@ public Partition apply(com.amazonaws.services.glue.model.Partition gluePartition private static final class StorageConverter { private final Function, List> bucketColumns = memoizeLast(ImmutableList::copyOf); - private final Function, List> sortColumns = memoizeLast(StorageConverter::createSortingColumns); + private final Function, List> sortColumns = memoizeLast(StorageConverter::createSortingColumns); private final UnaryOperator> bucketProperty = memoizeLast(); private final Function, Map> serdeParametersConverter = parametersConverter(); private final StorageFormatConverter storageFormatConverter = new StorageFormatConverter(); public void setStorageBuilder(StorageDescriptor sd, Storage.Builder storageBuilder, Map tableParameters) { - requireNonNull(sd.getSerdeInfo(), "StorageDescriptor SerDeInfo is null"); - SerDeInfo serdeInfo = sd.getSerdeInfo(); + requireNonNull(sd.serdeInfo(), "StorageDescriptor SerDeInfo is null"); + SerDeInfo serdeInfo = sd.serdeInfo(); storageBuilder.setStorageFormat(storageFormatConverter.createStorageFormat(serdeInfo, sd)) - .setLocation(nullToEmpty(sd.getLocation())) + .setLocation(nullToEmpty(sd.location())) .setBucketProperty(convertToBucketProperty(tableParameters, sd)) - .setSkewed(sd.getSkewedInfo() != null && !isNullOrEmpty(sd.getSkewedInfo().getSkewedColumnNames())) + .setSkewed(sd.skewedInfo() != null && !isNullOrEmpty(sd.skewedInfo().skewedColumnNames())) .setSerdeParameters(serdeParametersConverter.apply(getSerDeInfoParameters(serdeInfo))) .build(); } private Optional convertToBucketProperty(Map tableParameters, StorageDescriptor sd) { - if (sd.getNumberOfBuckets() > 0) { - if (isNullOrEmpty(sd.getBucketColumns())) { + if (sd.numberOfBuckets() > 0) { + if (isNullOrEmpty(sd.bucketColumns())) { throw new TrinoException(HIVE_INVALID_METADATA, "Table/partition metadata has 'numBuckets' set, but 'bucketCols' is not set"); } - List bucketColumns = this.bucketColumns.apply(sd.getBucketColumns()); - List sortedBy = this.sortColumns.apply(sd.getSortColumns()); + List bucketColumns = this.bucketColumns.apply(sd.bucketColumns()); + List sortedBy = this.sortColumns.apply(sd.sortColumns()); BucketingVersion bucketingVersion = HiveBucketing.getBucketingVersion(tableParameters); - return bucketProperty.apply(Optional.of(new HiveBucketProperty(bucketColumns, bucketingVersion, sd.getNumberOfBuckets(), sortedBy))); + return bucketProperty.apply(Optional.of(new HiveBucketProperty(bucketColumns, bucketingVersion, sd.numberOfBuckets(), sortedBy))); } return Optional.empty(); } - private static List createSortingColumns(List sortColumns) + private static List createSortingColumns(List sortColumns) { if (isNullOrEmpty(sortColumns)) { return ImmutableList.of(); } - return mappedCopy(sortColumns, column -> new SortingColumn(column.getColumn(), Order.fromMetastoreApiOrder(column.getSortOrder(), "unknown"))); + return mappedCopy(sortColumns, column -> new SortingColumn(column.column(), Order.fromMetastoreApiOrder(column.sortOrder(), "unknown"))); } } @@ -291,9 +291,9 @@ private static final class StorageFormatConverter public StorageFormat createStorageFormat(SerDeInfo serdeInfo, StorageDescriptor storageDescriptor) { - String serializationLib = this.serializationLib.apply(serdeInfo.getSerializationLibrary()); - String inputFormat = this.inputFormat.apply(storageDescriptor.getInputFormat()); - String outputFormat = this.outputFormat.apply(storageDescriptor.getOutputFormat()); + String serializationLib = this.serializationLib.apply(serdeInfo.serializationLibrary()); + String inputFormat = this.inputFormat.apply(storageDescriptor.inputFormat()); + String outputFormat = this.outputFormat.apply(storageDescriptor.outputFormat()); if (serializationLib == null && inputFormat == null && outputFormat == null) { return ALL_NULLS; } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreConfig.java index a5c6a4472fa8..f1a123944f7d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastoreConfig.java @@ -33,7 +33,6 @@ public void testDefaults() .setGlueStsRegion(null) .setGlueStsEndpointUrl(null) .setGlueProxyApiId(null) - .setPinGlueClientToCurrentRegion(false) .setMaxGlueConnections(30) .setMaxGlueErrorRetries(10) .setDefaultWarehouseDir(null) @@ -59,7 +58,6 @@ public void testExplicitPropertyMapping() .put("hive.metastore.glue.sts.region", "us-east-3") .put("hive.metastore.glue.sts.endpoint", "http://sts.foo.bar") .put("hive.metastore.glue.proxy-api-id", "abc123") - .put("hive.metastore.glue.pin-client-to-current-region", "true") .put("hive.metastore.glue.max-connections", "10") .put("hive.metastore.glue.max-error-retries", "20") .put("hive.metastore.glue.default-warehouse-dir", "/location") @@ -82,7 +80,6 @@ public void testExplicitPropertyMapping() .setGlueStsRegion("us-east-3") .setGlueStsEndpointUrl("http://sts.foo.bar") .setGlueProxyApiId("abc123") - .setPinGlueClientToCurrentRegion(true) .setMaxGlueConnections(10) .setMaxGlueErrorRetries(20) .setDefaultWarehouseDir("/location") diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueInputConverter.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueInputConverter.java index 9b812db8b8cc..96cb7b18c57c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueInputConverter.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueInputConverter.java @@ -13,10 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.model.DatabaseInput; -import com.amazonaws.services.glue.model.PartitionInput; -import com.amazonaws.services.glue.model.StorageDescriptor; -import com.amazonaws.services.glue.model.TableInput; import com.google.common.collect.ImmutableList; import io.trino.plugin.hive.HiveBucketProperty; import io.trino.plugin.hive.metastore.Column; @@ -26,6 +22,10 @@ import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.glue.converter.GlueInputConverter; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.PartitionInput; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.TableInput; import java.util.List; @@ -46,10 +46,10 @@ public void testConvertDatabase() { DatabaseInput dbInput = GlueInputConverter.convertDatabase(testDb); - assertEquals(dbInput.getName(), testDb.getDatabaseName()); - assertEquals(dbInput.getDescription(), testDb.getComment().get()); - assertEquals(dbInput.getLocationUri(), testDb.getLocation().get()); - assertEquals(dbInput.getParameters(), testDb.getParameters()); + assertEquals(dbInput.name(), testDb.getDatabaseName()); + assertEquals(dbInput.description(), testDb.getComment().get()); + assertEquals(dbInput.locationUri(), testDb.getLocation().get()); + assertEquals(dbInput.parameters(), testDb.getParameters()); } @Test @@ -57,15 +57,15 @@ public void testConvertTable() { TableInput tblInput = GlueInputConverter.convertTable(testTbl); - assertEquals(tblInput.getName(), testTbl.getTableName()); - assertEquals(tblInput.getOwner(), testTbl.getOwner().orElse(null)); - assertEquals(tblInput.getTableType(), testTbl.getTableType()); - assertEquals(tblInput.getParameters(), testTbl.getParameters()); - assertColumnList(tblInput.getStorageDescriptor().getColumns(), testTbl.getDataColumns()); - assertColumnList(tblInput.getPartitionKeys(), testTbl.getPartitionColumns()); - assertStorage(tblInput.getStorageDescriptor(), testTbl.getStorage()); - assertEquals(tblInput.getViewExpandedText(), testTbl.getViewExpandedText().get()); - assertEquals(tblInput.getViewOriginalText(), testTbl.getViewOriginalText().get()); + assertEquals(tblInput.name(), testTbl.getTableName()); + assertEquals(tblInput.owner(), testTbl.getOwner().orElse(null)); + assertEquals(tblInput.tableType(), testTbl.getTableType()); + assertEquals(tblInput.parameters(), testTbl.getParameters()); + assertColumnList(tblInput.storageDescriptor().columns(), testTbl.getDataColumns()); + assertColumnList(tblInput.partitionKeys(), testTbl.getPartitionColumns()); + assertStorage(tblInput.storageDescriptor(), testTbl.getStorage()); + assertEquals(tblInput.viewExpandedText(), testTbl.getViewExpandedText().get()); + assertEquals(tblInput.viewOriginalText(), testTbl.getViewOriginalText().get()); } @Test @@ -73,12 +73,12 @@ public void testConvertPartition() { PartitionInput partitionInput = GlueInputConverter.convertPartition(testPartition); - assertEquals(partitionInput.getParameters(), testPartition.getParameters()); - assertStorage(partitionInput.getStorageDescriptor(), testPartition.getStorage()); - assertEquals(partitionInput.getValues(), testPartition.getValues()); + assertEquals(partitionInput.parameters(), testPartition.getParameters()); + assertStorage(partitionInput.storageDescriptor(), testPartition.getStorage()); + assertEquals(partitionInput.values(), testPartition.getValues()); } - private static void assertColumnList(List actual, List expected) + private static void assertColumnList(List actual, List expected) { if (expected == null) { assertNull(actual); @@ -90,24 +90,24 @@ private static void assertColumnList(List) null).build(); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertTrue(trinoTable.getPartitionColumns().isEmpty()); } @Test public void testConvertTableUppercaseColumnType() { - com.amazonaws.services.glue.model.Column uppercaseColumn = getGlueTestColumn().withType("String"); - testTable.getStorageDescriptor().setColumns(ImmutableList.of(uppercaseColumn)); - GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + software.amazon.awssdk.services.glue.model.Column uppercaseColumn = getGlueTestColumn().toBuilder().type("String").build(); + StorageDescriptor sd = testTable.storageDescriptor(); + Table table = testTable.toBuilder() + .storageDescriptor(sd.toBuilder() + .columns(ImmutableList.of(uppercaseColumn)) + .build()) + .build(); + GlueToTrinoConverter.convertTable(table, testDatabase.name()); } @Test @@ -160,11 +169,11 @@ public void testConvertPartition() { GluePartitionConverter converter = createPartitionConverter(testTable); io.trino.plugin.hive.metastore.Partition trinoPartition = converter.apply(testPartition); - assertEquals(trinoPartition.getDatabaseName(), testPartition.getDatabaseName()); - assertEquals(trinoPartition.getTableName(), testPartition.getTableName()); - assertColumnList(trinoPartition.getColumns(), testPartition.getStorageDescriptor().getColumns()); - assertEquals(trinoPartition.getValues(), testPartition.getValues()); - assertStorage(trinoPartition.getStorage(), testPartition.getStorageDescriptor()); + assertEquals(trinoPartition.getDatabaseName(), testPartition.databaseName()); + assertEquals(trinoPartition.getTableName(), testPartition.tableName()); + assertColumnList(trinoPartition.getColumns(), testPartition.storageDescriptor().columns()); + assertEquals(trinoPartition.getValues(), testPartition.values()); + assertStorage(trinoPartition.getStorage(), testPartition.storageDescriptor()); assertEquals(trinoPartition.getParameters(), getPartitionParameters(testPartition)); } @@ -172,19 +181,33 @@ public void testConvertPartition() public void testPartitionConversionMemoization() { String fakeS3Location = "s3://some-fake-location"; - testPartition.getStorageDescriptor().setLocation(fakeS3Location); + StorageDescriptor sd = testPartition.storageDescriptor(); + Partition partition = testPartition.toBuilder() + .storageDescriptor(sd.toBuilder() + .location(fakeS3Location) + .build()) + .build(); // Second partition to convert with equal (but not aliased) values - Partition partitionTwo = getGlueTestPartition("" + testDatabase.getName(), "" + testTable.getName(), new ArrayList<>(testPartition.getValues())); + Partition partitionTwo = getGlueTestPartition( + "" + testDatabase.name(), + "" + testTable.name(), + new ArrayList<>(testPartition.values())); // Ensure storage fields match as well - partitionTwo.getStorageDescriptor().setColumns(new ArrayList<>(testPartition.getStorageDescriptor().getColumns())); - partitionTwo.getStorageDescriptor().setBucketColumns(new ArrayList<>(testPartition.getStorageDescriptor().getBucketColumns())); - partitionTwo.getStorageDescriptor().setLocation("" + fakeS3Location); - partitionTwo.getStorageDescriptor().setInputFormat("" + testPartition.getStorageDescriptor().getInputFormat()); - partitionTwo.getStorageDescriptor().setOutputFormat("" + testPartition.getStorageDescriptor().getOutputFormat()); - partitionTwo.getStorageDescriptor().setParameters(new HashMap<>(testPartition.getStorageDescriptor().getParameters())); + partitionTwo = partitionTwo.toBuilder().storageDescriptor( + StorageDescriptor.builder() + .columns(new ArrayList<>(sd.columns())) + .bucketColumns(new ArrayList<>(sd.bucketColumns())) + .serdeInfo(partitionTwo.storageDescriptor().serdeInfo()) + .location("" + fakeS3Location) + .inputFormat("" + sd.inputFormat()) + .outputFormat("" + sd.outputFormat()) + .parameters(new HashMap<>(sd.parameters())) + .numberOfBuckets(partitionTwo.storageDescriptor().numberOfBuckets()) + .build()) + .build(); GluePartitionConverter converter = createPartitionConverter(testTable); - io.trino.plugin.hive.metastore.Partition trinoPartition = converter.apply(testPartition); + io.trino.plugin.hive.metastore.Partition trinoPartition = converter.apply(partition); io.trino.plugin.hive.metastore.Partition trinoPartition2 = converter.apply(partitionTwo); assertNotSame(trinoPartition, trinoPartition2); @@ -206,16 +229,24 @@ public void testPartitionConversionMemoization() @Test public void testDatabaseNullParameters() { - testDatabase.setParameters(null); - assertNotNull(GlueToTrinoConverter.convertDatabase(testDatabase).getParameters()); + Database database = testDatabase.toBuilder().parameters(null).build(); + assertNotNull(GlueToTrinoConverter.convertDatabase(database).getParameters()); } @Test public void testTableNullParameters() { - testTable.setParameters(null); - testTable.getStorageDescriptor().getSerdeInfo().setParameters(null); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + StorageDescriptor sd = testTable.storageDescriptor(); + SerDeInfo serDeInfo = sd.serdeInfo(); + Table table = testTable.toBuilder().parameters(null).storageDescriptor( + sd.toBuilder().serdeInfo( + serDeInfo.toBuilder() + .parameters(null) + .build()) + .build()) + .build(); + + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertNotNull(trinoTable.getParameters()); assertNotNull(trinoTable.getStorage().getSerdeParameters()); } @@ -223,62 +254,66 @@ public void testTableNullParameters() @Test public void testIcebergTableNullStorageDescriptor() { - testTable.setParameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)); - testTable.setStorageDescriptor(null); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + Table table = testTable.toBuilder() + .parameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)) + .storageDescriptor((StorageDescriptor) null) + .build(); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertEquals(trinoTable.getDataColumns().size(), 1); } @Test public void testIcebergTableNonNullStorageDescriptor() { - testTable.setParameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)); - assertNotNull(testTable.getStorageDescriptor()); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + Table table = testTable.toBuilder().parameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)).build(); + assertNotNull(testTable.storageDescriptor()); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertEquals(trinoTable.getDataColumns().size(), 1); } @Test public void testDeltaTableNullStorageDescriptor() { - testTable.setParameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)); - testTable.setStorageDescriptor(null); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + Table table = testTable.toBuilder() + .parameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)) + .storageDescriptor((StorageDescriptor) null) + .build(); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertEquals(trinoTable.getDataColumns().size(), 1); } @Test public void testDeltaTableNonNullStorageDescriptor() { - testTable.setParameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)); - assertNotNull(testTable.getStorageDescriptor()); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testTable, testDatabase.getName()); + Table table = testTable.toBuilder().parameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)).build(); + assertNotNull(testTable.storageDescriptor()); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(table, testDatabase.name()); assertEquals( trinoTable.getDataColumns().stream() .map(Column::getName) .collect(toImmutableSet()), - testTable.getStorageDescriptor().getColumns().stream() - .map(com.amazonaws.services.glue.model.Column::getName) + table.storageDescriptor().columns().stream() + .map(software.amazon.awssdk.services.glue.model.Column::name) .collect(toImmutableSet())); } @Test public void testIcebergMaterializedViewNullStorageDescriptor() { - Table testMaterializedView = getGlueTestTrinoMaterializedView(testDatabase.getName()); - assertNull(testMaterializedView.getStorageDescriptor()); - io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testMaterializedView, testDatabase.getName()); + Table testMaterializedView = getGlueTestTrinoMaterializedView(testDatabase.name()); + assertNull(testMaterializedView.storageDescriptor()); + io.trino.plugin.hive.metastore.Table trinoTable = GlueToTrinoConverter.convertTable(testMaterializedView, testDatabase.name()); assertEquals(trinoTable.getDataColumns().size(), 1); } @Test public void testPartitionNullParameters() { - testPartition.setParameters(null); - assertNotNull(createPartitionConverter(testTable).apply(testPartition).getParameters()); + Partition partition = testPartition.toBuilder().parameters(null).build(); + assertNotNull(createPartitionConverter(testTable).apply(partition).getParameters()); } - private static void assertColumnList(List actual, List expected) + private static void assertColumnList(List actual, List expected) { if (expected == null) { assertNull(actual); @@ -290,23 +325,23 @@ private static void assertColumnList(List actual, List orphanedDatabases = getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - stats.getGetDatabases()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .filter(database -> database.getName().startsWith(TEST_DATABASE_NAME_PREFIX) && - database.getCreateTime().getTime() <= creationTimeMillisThreshold) - .map(Database::getName) - .collect(toImmutableList()); + + ImmutableList.Builder orphanedDbBuilder = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getDatabasesPaginator(GetDatabasesRequest.builder().build()), + getDatabasesResponse -> getDatabasesResponse.databaseList() + .stream() + .filter(database -> database.name().startsWith(TEST_DATABASE_NAME_PREFIX) + && database.createTime().toEpochMilli() <= creationTimeMillisThreshold) + .forEach(database -> { + orphanedDbBuilder.add(database.name()); + }), + stats.getGetDatabases()); + + List orphanedDatabases = orphanedDbBuilder.build(); log.info("Found %s %s* databases that look orphaned, removing", orphanedDatabases.size(), TEST_DATABASE_NAME_PREFIX); orphanedDatabases.forEach(database -> { try { - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(database)); + awsSyncRequest(glueClient::deleteDatabase, DeleteDatabaseRequest.builder() + .name(database) + .build(), null); } catch (EntityNotFoundException e) { log.info("Database [%s] not found, could be removed by other cleanup process", database); @@ -1325,14 +1338,13 @@ public void testInvalidColumnStatisticsMetadata() Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).get(); TableInput tableInput = GlueInputConverter.convertTable(table); - tableInput.setParameters(ImmutableMap.builder() - .putAll(tableInput.getParameters()) + tableInput = tableInput.toBuilder().parameters(ImmutableMap.builder() + .putAll(tableInput.parameters()) .put("column_stats_bad_data", "bad data") - .buildOrThrow()); - getGlueClient().updateTable(new UpdateTableRequest() - .withDatabaseName(tableName.getSchemaName()) - .withTableInput(tableInput)); - + .buildOrThrow()).build(); + awsSyncRequest(getGlueClient()::updateTable, UpdateTableRequest.builder() + .databaseName(tableName.getSchemaName()) + .tableInput(tableInput).build(), null); assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName())) .isEqualTo(partitionStatistics); } @@ -1346,51 +1358,52 @@ public void testGlueObjectsWithoutStorageDescriptor() { // StorageDescriptor is an Optional field for Glue tables. SchemaTableName table = temporaryTable("test_missing_storage_descriptor"); - DeleteTableRequest deleteTableRequest = new DeleteTableRequest() - .withDatabaseName(table.getSchemaName()) - .withName(table.getTableName()); + DeleteTableRequest deleteTableRequest = DeleteTableRequest.builder() + .databaseName(table.getSchemaName()) + .name(table.getTableName()) + .build(); try { - Supplier resetTableInput = () -> new TableInput() - .withStorageDescriptor(null) - .withName(table.getTableName()) - .withTableType(EXTERNAL_TABLE.name()); + Supplier resetTableInput = () -> TableInput.builder() + .storageDescriptor((StorageDescriptor) null) + .name(table.getTableName()) + .tableType(EXTERNAL_TABLE.name()); - TableInput tableInput = resetTableInput.get(); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); + TableInput tableInput = resetTableInput.get().build(); + awsSyncRequest(glueClient::createTable, CreateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput).build(), null); assertThatThrownBy(() -> metastore.getTable(table.getSchemaName(), table.getTableName())) .hasMessageStartingWith("Table StorageDescriptor is null for table"); - glueClient.deleteTable(deleteTableRequest); + awsSyncRequest(glueClient::deleteTable, deleteTableRequest, null); // Iceberg table - tableInput = resetTableInput.get().withParameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); + tableInput = resetTableInput.get().parameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)).build(); + awsSyncRequest(glueClient::createTable, CreateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput).build(), null); assertTrue(isIcebergTable(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())); - glueClient.deleteTable(deleteTableRequest); + awsSyncRequest(glueClient::deleteTable, deleteTableRequest, null); // Delta Lake table - tableInput = resetTableInput.get().withParameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); + tableInput = resetTableInput.get().parameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)).build(); + awsSyncRequest(glueClient::createTable, CreateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput).build(), null); assertTrue(isDeltaLakeTable(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())); - glueClient.deleteTable(deleteTableRequest); + awsSyncRequest(glueClient::deleteTable, deleteTableRequest, null); // Iceberg materialized view - tableInput = resetTableInput.get().withTableType(VIRTUAL_VIEW.name()) - .withViewOriginalText("/* Presto Materialized View: eyJvcmlnaW5hbFNxbCI6IlNFTEVDVCAxIiwiY29sdW1ucyI6W3sibmFtZSI6ImEiLCJ0eXBlIjoiaW50ZWdlciJ9XX0= */") - .withViewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT) - .withParameters(ImmutableMap.of( + tableInput = resetTableInput.get().tableType(VIRTUAL_VIEW.name()) + .viewOriginalText("/* Presto Materialized View: eyJvcmlnaW5hbFNxbCI6IlNFTEVDVCAxIiwiY29sdW1ucyI6W3sibmFtZSI6ImEiLCJ0eXBlIjoiaW50ZWdlciJ9XX0= */") + .viewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT) + .parameters(ImmutableMap.of( PRESTO_VIEW_FLAG, "true", - TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); + TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT)).build(); + awsSyncRequest(glueClient::createTable, CreateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput).build(), null); assertTrue(isTrinoMaterializedView(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())); materializedViews.add(table); try (Transaction transaction = newTransaction()) { @@ -1409,9 +1422,10 @@ public void testGlueObjectsWithoutStorageDescriptor() } finally { // Table cannot be dropped through HiveMetastore since a TableHandle cannot be created - glueClient.deleteTable(new DeleteTableRequest() - .withDatabaseName(table.getSchemaName()) - .withName(table.getTableName())); + awsSyncRequest(glueClient::deleteTable, DeleteTableRequest.builder() + .databaseName(table.getSchemaName()) + .name(table.getTableName()) + .build(), null); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestingMetastoreObjects.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestingMetastoreObjects.java index 0594cbf8814b..4a1dc93de969 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestingMetastoreObjects.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestingMetastoreObjects.java @@ -13,12 +13,6 @@ */ package io.trino.plugin.hive.metastore.glue; -import com.amazonaws.services.glue.model.Column; -import com.amazonaws.services.glue.model.Database; -import com.amazonaws.services.glue.model.Partition; -import com.amazonaws.services.glue.model.SerDeInfo; -import com.amazonaws.services.glue.model.StorageDescriptor; -import com.amazonaws.services.glue.model.Table; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.HiveType; @@ -26,6 +20,12 @@ import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.spi.security.PrincipalType; import org.apache.hadoop.hive.metastore.TableType; +import software.amazon.awssdk.services.glue.model.Column; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.Partition; +import software.amazon.awssdk.services.glue.model.SerDeInfo; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.Table; import java.util.List; import java.util.Optional; @@ -45,39 +45,41 @@ private TestingMetastoreObjects() {} public static Database getGlueTestDatabase() { - return new Database() - .withName("test-db" + generateRandom()) - .withDescription("database desc") - .withLocationUri("/db") - .withParameters(ImmutableMap.of()); + return Database.builder() + .name("test-db" + generateRandom()) + .description("database desc") + .locationUri("/db") + .parameters(ImmutableMap.of()) + .build(); } public static Table getGlueTestTable(String dbName) { - return new Table() - .withDatabaseName(dbName) - .withName("test-tbl" + generateRandom()) - .withOwner("owner") - .withParameters(ImmutableMap.of()) - .withPartitionKeys(ImmutableList.of(getGlueTestColumn())) - .withStorageDescriptor(getGlueTestStorageDescriptor()) - .withTableType(TableType.EXTERNAL_TABLE.name()) - .withViewOriginalText("originalText") - .withViewExpandedText("expandedText"); + return Table.builder() + .databaseName(dbName) + .name("test-tbl" + generateRandom()) + .owner("owner") + .parameters(ImmutableMap.of()) + .partitionKeys(ImmutableList.of(getGlueTestColumn())) + .storageDescriptor(getGlueTestStorageDescriptor()) + .tableType(TableType.EXTERNAL_TABLE.name()) + .viewOriginalText("originalText") + .viewExpandedText("expandedText") + .build(); } public static Table getGlueTestTrinoMaterializedView(String dbName) { - return new Table() - .withDatabaseName(dbName) - .withName("test-mv" + generateRandom()) - .withOwner("owner") - .withParameters(ImmutableMap.of(PRESTO_VIEW_FLAG, "true", TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT)) - .withPartitionKeys() - .withStorageDescriptor(null) - .withTableType(TableType.VIRTUAL_VIEW.name()) - .withViewOriginalText("/* %s: base64encodedquery */".formatted(ICEBERG_MATERIALIZED_VIEW_COMMENT)) - .withViewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT); + return Table.builder() + .databaseName(dbName) + .name("test-mv" + generateRandom()) + .owner("owner") + .parameters(ImmutableMap.of(PRESTO_VIEW_FLAG, "true", TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT)) + .partitionKeys(ImmutableList.of()) + .storageDescriptor((StorageDescriptor) null) + .tableType(TableType.VIRTUAL_VIEW.name()) + .viewOriginalText("/* %s: base64encodedquery */".formatted(ICEBERG_MATERIALIZED_VIEW_COMMENT)) + .viewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT).build(); } public static Column getGlueTestColumn() @@ -87,10 +89,11 @@ public static Column getGlueTestColumn() public static Column getGlueTestColumn(String type) { - return new Column() - .withName("test-col" + generateRandom()) - .withType(type) - .withComment("column comment"); + return Column.builder() + .name("test-col" + generateRandom()) + .type(type) + .comment("column comment") + .build(); } public static StorageDescriptor getGlueTestStorageDescriptor() @@ -100,27 +103,30 @@ public static StorageDescriptor getGlueTestStorageDescriptor() public static StorageDescriptor getGlueTestStorageDescriptor(List columns, String serde) { - return new StorageDescriptor() - .withBucketColumns(ImmutableList.of("test-bucket-col")) - .withColumns(columns) - .withParameters(ImmutableMap.of()) - .withSerdeInfo(new SerDeInfo() - .withSerializationLibrary(serde) - .withParameters(ImmutableMap.of())) - .withInputFormat("InputFormat") - .withOutputFormat("OutputFormat") - .withLocation("/test-tbl") - .withNumberOfBuckets(1); + return StorageDescriptor.builder() + .bucketColumns(ImmutableList.of("test-bucket-col")) + .columns(columns) + .parameters(ImmutableMap.of()) + .serdeInfo(SerDeInfo.builder() + .serializationLibrary(serde) + .parameters(ImmutableMap.of()) + .build()) + .inputFormat("InputFormat") + .outputFormat("OutputFormat") + .location("/test-tbl") + .numberOfBuckets(1) + .build(); } public static Partition getGlueTestPartition(String dbName, String tblName, List values) { - return new Partition() - .withDatabaseName(dbName) - .withTableName(tblName) - .withValues(values) - .withParameters(ImmutableMap.of()) - .withStorageDescriptor(getGlueTestStorageDescriptor()); + return Partition.builder() + .databaseName(dbName) + .tableName(tblName) + .values(values) + .parameters(ImmutableMap.of()) + .storageDescriptor(getGlueTestStorageDescriptor()) + .build(); } // --------------- Trino Objects --------------- diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 7b7ba2474112..985297a99f1c 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -29,16 +29,6 @@ - - com.amazonaws - aws-java-sdk-core - - - - com.amazonaws - aws-java-sdk-glue - - com.fasterxml.jackson.core jackson-core @@ -261,6 +251,36 @@ jmxutils + + software.amazon.awssdk + auth + + + + software.amazon.awssdk + aws-core + + + + software.amazon.awssdk + glue + + + + software.amazon.awssdk + metrics-spi + + + + software.amazon.awssdk + sdk-core + + + + software.amazon.awssdk + utils + + com.fasterxml.jackson.core jackson-annotations @@ -552,6 +572,21 @@ org.antlr antlr4-maven-plugin + + org.apache.maven.plugins + maven-dependency-plugin + + + software.amazon.awssdk:metrics-spi + + + + software.amazon.awssdk:utils + + + org.basepom.maven duplicate-finder-maven-plugin @@ -562,6 +597,17 @@ + + maven-dependency-plugin + + + + software.amazon.awssdk:utils + + + org.apache.maven.plugins maven-surefire-plugin diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java index 907a238fa0d0..159d9f00bed6 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java @@ -13,17 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.AlreadyExistsException; -import com.amazonaws.services.glue.model.ConcurrentModificationException; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.InvalidInputException; -import com.amazonaws.services.glue.model.ResourceNumberLimitExceededException; -import com.amazonaws.services.glue.model.Table; -import com.amazonaws.services.glue.model.TableInput; -import com.amazonaws.services.glue.model.UpdateTableRequest; import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; import io.trino.plugin.iceberg.UnknownTableTypeException; @@ -36,6 +25,17 @@ import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.io.FileIO; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.ConcurrentModificationException; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.InvalidInputException; +import software.amazon.awssdk.services.glue.model.ResourceNumberLimitExceededException; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; import java.util.Map; import java.util.Optional; @@ -56,14 +56,14 @@ public class GlueIcebergTableOperations extends AbstractIcebergTableOperations { - private final AWSGlueAsync glueClient; + private final GlueClient glueClient; private final GlueMetastoreStats stats; @Nullable private String glueVersionId; protected GlueIcebergTableOperations( - AWSGlueAsync glueClient, + GlueClient glueClient, GlueMetastoreStats stats, FileIO fileIo, ConnectorSession session, @@ -81,7 +81,7 @@ protected GlueIcebergTableOperations( protected String getRefreshedLocation(boolean invalidateCaches) { Table table = getTable(); - glueVersionId = table.getVersionId(); + glueVersionId = table.versionId(); Map parameters = getTableParameters(table); if (isPrestoView(parameters) && isHiveOrPrestoView(getTableType(table))) { @@ -106,11 +106,13 @@ protected void commitNewTable(TableMetadata metadata) String newMetadataLocation = writeNewMetadata(metadata, 0); TableInput tableInput = getTableInput(tableName, owner, ImmutableMap.of(METADATA_LOCATION_PROP, newMetadataLocation)); - CreateTableRequest createTableRequest = new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput); + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput) + .build(); try { - stats.getCreateTable().call(() -> glueClient.createTable(createTableRequest)); + stats.getCreateTable().call(() -> + glueClient.createTable(createTableRequest)); } catch (AlreadyExistsException | EntityNotFoundException @@ -134,12 +136,14 @@ protected void commitToExistingTable(TableMetadata base, TableMetadata metadata) METADATA_LOCATION_PROP, newMetadataLocation, PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation)); - UpdateTableRequest updateTableRequest = new UpdateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput) - .withVersionId(glueVersionId); + UpdateTableRequest updateTableRequest = UpdateTableRequest.builder() + .databaseName(database) + .tableInput(tableInput) + .versionId(glueVersionId) + .build(); try { - stats.getUpdateTable().call(() -> glueClient.updateTable(updateTableRequest)); + stats.getUpdateTable().call(() -> + glueClient.updateTable(updateTableRequest)); } catch (ConcurrentModificationException e) { // CommitFailedException is handled as a special case in the Iceberg library. This commit will automatically retry @@ -160,10 +164,13 @@ protected void commitToExistingTable(TableMetadata base, TableMetadata metadata) private Table getTable() { try { - GetTableRequest getTableRequest = new GetTableRequest() - .withDatabaseName(database) - .withName(tableName); - return stats.getGetTable().call(() -> glueClient.getTable(getTableRequest).getTable()); + GetTableRequest getTableRequest = GetTableRequest.builder() + .databaseName(database) + .name(tableName) + .build(); + return stats.getGetTable().call( + () -> glueClient.getTable(getTableRequest) + ).table(); } catch (EntityNotFoundException e) { throw new TableNotFoundException(getSchemaTableName(), e); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java index b82b54cf33e9..a597dc94a763 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.inject.Inject; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; @@ -22,6 +21,7 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; import io.trino.spi.connector.ConnectorSession; +import software.amazon.awssdk.services.glue.GlueClient; import java.util.Optional; @@ -31,14 +31,14 @@ public class GlueIcebergTableOperationsProvider implements IcebergTableOperationsProvider { private final TrinoFileSystemFactory fileSystemFactory; - private final AWSGlueAsync glueClient; + private final GlueClient glueClient; private final GlueMetastoreStats stats; @Inject public GlueIcebergTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, GlueMetastoreStats stats, - AWSGlueAsync glueClient) + GlueClient glueClient) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.stats = requireNonNull(stats, "stats is null"); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergUtil.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergUtil.java index 06a9b0c97f00..e603f32ee6f9 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergUtil.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergUtil.java @@ -13,9 +13,9 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.model.TableInput; import com.google.common.collect.ImmutableMap; import jakarta.annotation.Nullable; +import software.amazon.awssdk.services.glue.model.TableInput; import java.util.Map; import java.util.Optional; @@ -34,36 +34,39 @@ private GlueIcebergUtil() {} public static TableInput getTableInput(String tableName, Optional owner, Map parameters) { - return new TableInput() - .withName(tableName) - .withOwner(owner.orElse(null)) - .withParameters(ImmutableMap.builder() + return TableInput.builder() + .name(tableName) + .owner(owner.orElse(null)) + .parameters(ImmutableMap.builder() .putAll(parameters) .put(TABLE_TYPE_PROP, ICEBERG_TABLE_TYPE_VALUE.toUpperCase(ENGLISH)) .buildKeepingLast()) // Iceberg does not distinguish managed and external tables, all tables are treated the same and marked as EXTERNAL - .withTableType(EXTERNAL_TABLE.name()); + .tableType(EXTERNAL_TABLE.name()) + .build(); } public static TableInput getViewTableInput(String viewName, String viewOriginalText, @Nullable String owner, Map parameters) { - return new TableInput() - .withName(viewName) - .withTableType(VIRTUAL_VIEW.name()) - .withViewOriginalText(viewOriginalText) - .withViewExpandedText(PRESTO_VIEW_EXPANDED_TEXT_MARKER) - .withOwner(owner) - .withParameters(parameters); + return TableInput.builder() + .name(viewName) + .tableType(VIRTUAL_VIEW.name()) + .viewOriginalText(viewOriginalText) + .viewExpandedText(PRESTO_VIEW_EXPANDED_TEXT_MARKER) + .owner(owner) + .parameters(parameters) + .build(); } public static TableInput getMaterializedViewTableInput(String viewName, String viewOriginalText, String owner, Map parameters) { - return new TableInput() - .withName(viewName) - .withTableType(VIRTUAL_VIEW.name()) - .withViewOriginalText(viewOriginalText) - .withViewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT) - .withOwner(owner) - .withParameters(parameters); + return TableInput.builder() + .name(viewName) + .tableType(VIRTUAL_VIEW.name()) + .viewOriginalText(viewOriginalText) + .viewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT) + .owner(owner) + .parameters(parameters) + .build(); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/IcebergGlueCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/IcebergGlueCatalogModule.java index 1df3e42ca5cd..767b09f5e748 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/IcebergGlueCatalogModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/IcebergGlueCatalogModule.java @@ -13,9 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Provides; @@ -34,6 +31,9 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; import io.trino.plugin.iceberg.procedure.MigrateProcedure; import io.trino.spi.procedure.Procedure; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.model.Table; import java.util.function.Predicate; @@ -52,7 +52,7 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(IcebergGlueCatalogConfig.class); binder.bind(GlueMetastoreStats.class).in(Scopes.SINGLETON); newExporter(binder).export(GlueMetastoreStats.class).withGeneratedName(); - binder.bind(AWSCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); + binder.bind(AwsCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); binder.bind(IcebergTableOperationsProvider.class).to(GlueIcebergTableOperationsProvider.class).in(Scopes.SINGLETON); binder.bind(TrinoCatalogFactory.class).to(TrinoGlueCatalogFactory.class).in(Scopes.SINGLETON); newExporter(binder).export(TrinoCatalogFactory.class).withGeneratedName(); @@ -69,7 +69,7 @@ protected void setup(Binder binder) @Provides @Singleton @ForGlueHiveMetastore - public static RequestHandler2 createRequestHandler(IcebergGlueCatalogConfig config) + public static ExecutionInterceptor createRequestHandler(IcebergGlueCatalogConfig config) { return new SkipArchiveRequestHandler(config.isSkipArchive()); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/SkipArchiveRequestHandler.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/SkipArchiveRequestHandler.java index 8fa3796fe45f..e73700cfd481 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/SkipArchiveRequestHandler.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/SkipArchiveRequestHandler.java @@ -13,20 +13,22 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.model.CreateDatabaseRequest; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.GetDatabaseRequest; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.GetTablesRequest; -import com.amazonaws.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; public class SkipArchiveRequestHandler - extends RequestHandler2 + implements ExecutionInterceptor { private final boolean skipArchive; @@ -36,10 +38,11 @@ public SkipArchiveRequestHandler(boolean skipArchive) } @Override - public AmazonWebServiceRequest beforeExecution(AmazonWebServiceRequest request) + public SdkRequest modifyRequest(Context.ModifyRequest context, ExecutionAttributes executionAttributes) { + SdkRequest request = context.request(); if (request instanceof UpdateTableRequest updateTableRequest) { - return updateTableRequest.withSkipArchive(skipArchive); + return updateTableRequest.toBuilder().skipArchive(skipArchive).build(); } if (request instanceof CreateDatabaseRequest || request instanceof DeleteDatabaseRequest || diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java index a2e63378e892..6715fc8ffbf8 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java @@ -13,25 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.AccessDeniedException; -import com.amazonaws.services.glue.model.AlreadyExistsException; -import com.amazonaws.services.glue.model.CreateDatabaseRequest; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.Database; -import com.amazonaws.services.glue.model.DatabaseInput; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetDatabaseRequest; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetDatabasesResult; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.GetTablesRequest; -import com.amazonaws.services.glue.model.GetTablesResult; -import com.amazonaws.services.glue.model.TableInput; -import com.amazonaws.services.glue.model.UpdateTableRequest; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import dev.failsafe.Failsafe; @@ -69,6 +50,24 @@ import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.io.FileIO; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.AccessDeniedException; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.paginators.GetDatabasesIterable; import java.time.Duration; import java.util.List; @@ -89,7 +88,6 @@ import static io.trino.plugin.hive.ViewReaderUtil.encodeViewData; import static io.trino.plugin.hive.ViewReaderUtil.isPrestoView; import static io.trino.plugin.hive.ViewReaderUtil.isTrinoMaterializedView; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableParameters; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableType; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableTypeNullable; @@ -129,7 +127,7 @@ public class TrinoGlueCatalog private final String trinoVersion; private final TrinoFileSystemFactory fileSystemFactory; private final Optional defaultSchemaLocation; - private final AWSGlueAsync glueClient; + private final GlueClient glueClient; private final GlueMetastoreStats stats; private final Map tableMetadataCache = new ConcurrentHashMap<>(); @@ -142,7 +140,7 @@ public TrinoGlueCatalog( TypeManager typeManager, IcebergTableOperationsProvider tableOperationsProvider, String trinoVersion, - AWSGlueAsync glueClient, + GlueClient glueClient, GlueMetastoreStats stats, Optional defaultSchemaLocation, boolean useUniqueTableLocation) @@ -163,36 +161,32 @@ public boolean namespaceExists(ConnectorSession session, String namespace) // In fact, Glue stores database names lowercase only (but accepted mixed case on lookup). return false; } - return stats.getGetDatabase().call(() -> { - try { - glueClient.getDatabase(new GetDatabaseRequest().withName(namespace)); - return true; - } - catch (EntityNotFoundException e) { - return false; - } - catch (AmazonServiceException e) { - throw new TrinoException(ICEBERG_CATALOG_ERROR, e); - } - }); + try { + GetDatabaseRequest getDatabaseRequest = GetDatabaseRequest.builder().name(namespace).build(); + stats.getGetDatabase().call(() -> + glueClient.getDatabase(getDatabaseRequest)); + return true; + } + catch (EntityNotFoundException e) { + return false; + } + catch (AwsServiceException e) { + throw new TrinoException(ICEBERG_CATALOG_ERROR, e); + } } @Override public List listNamespaces(ConnectorSession session) { try { - return getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - stats.getGetDatabases()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .map(com.amazonaws.services.glue.model.Database::getName) - .collect(toImmutableList()); - } - catch (AmazonServiceException e) { + GetDatabasesIterable databasesPaginator = glueClient.getDatabasesPaginator(GetDatabasesRequest.builder().build()); + return stats.getGetDatabases().call(() -> + databasesPaginator.stream() + .flatMap(response -> response.databaseList().stream()) + .map(Database::name) + .collect(toImmutableList())); + } + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } } @@ -209,13 +203,14 @@ private List listNamespaces(ConnectorSession session, Optional n public void dropNamespace(ConnectorSession session, String namespace) { try { + DeleteDatabaseRequest deleteDatabaseRequest = DeleteDatabaseRequest.builder().name(namespace).build(); stats.getDeleteDatabase().call(() -> - glueClient.deleteDatabase(new DeleteDatabaseRequest().withName(namespace))); + glueClient.deleteDatabase(deleteDatabaseRequest)); } catch (EntityNotFoundException e) { throw new SchemaNotFoundException(namespace); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } } @@ -224,22 +219,22 @@ public void dropNamespace(ConnectorSession session, String namespace) public Map loadNamespaceMetadata(ConnectorSession session, String namespace) { try { - GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest().withName(namespace); + GetDatabaseRequest getDatabaseRequest = GetDatabaseRequest.builder().name(namespace).build(); Database database = stats.getGetDatabase().call(() -> - glueClient.getDatabase(getDatabaseRequest).getDatabase()); + glueClient.getDatabase(getDatabaseRequest).database()); ImmutableMap.Builder metadata = ImmutableMap.builder(); - if (database.getLocationUri() != null) { - metadata.put(LOCATION_PROPERTY, database.getLocationUri()); + if (database.locationUri() != null) { + metadata.put(LOCATION_PROPERTY, database.locationUri()); } - if (database.getParameters() != null) { - metadata.putAll(database.getParameters()); + if (database.parameters() != null) { + metadata.putAll(database.parameters()); } return metadata.buildOrThrow(); } catch (EntityNotFoundException e) { throw new SchemaNotFoundException(namespace); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } } @@ -257,29 +252,30 @@ public void createNamespace(ConnectorSession session, String namespace, Map - glueClient.createDatabase(new CreateDatabaseRequest() - .withDatabaseInput(createDatabaseInput(namespace, properties)))); + glueClient.createDatabase(createDatabaseRequest)); } catch (AlreadyExistsException e) { throw new SchemaAlreadyExistsException(namespace); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } } private DatabaseInput createDatabaseInput(String namespace, Map properties) { - DatabaseInput databaseInput = new DatabaseInput().withName(namespace); + DatabaseInput.Builder databaseInputBuilder = DatabaseInput.builder().name(namespace); properties.forEach((property, value) -> { switch (property) { - case LOCATION_PROPERTY -> databaseInput.setLocationUri((String) value); + case LOCATION_PROPERTY -> databaseInputBuilder.locationUri((String) value); default -> throw new IllegalArgumentException("Unrecognized property: " + property); } }); - return databaseInput; + return databaseInputBuilder.build(); } @Override @@ -302,25 +298,20 @@ public List listTables(ConnectorSession session, Optional namespaces = listNamespaces(session, namespace); for (String glueNamespace : namespaces) { try { - // Add all tables from a namespace together, in case it is removed while fetching paginated results - tables.addAll( - getPaginatedResults( - glueClient::getTables, - new GetTablesRequest().withDatabaseName(glueNamespace), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - stats.getGetTables()) - .map(GetTablesResult::getTableList) - .flatMap(List::stream) - .map(table -> new SchemaTableName(glueNamespace, table.getName())) + GetTablesRequest getTablesRequest = GetTablesRequest.builder().databaseName(glueNamespace).build(); + ImmutableList tablesInNamespace = stats.getGetTables().call(() -> + glueClient.getTablesPaginator(getTablesRequest).stream() + .flatMap(getTablesResponse -> getTablesResponse.tableList().stream()) + .map(table -> new SchemaTableName(glueNamespace, table.name())) .collect(toImmutableList())); + tables.addAll(tablesInNamespace); } catch (EntityNotFoundException | AccessDeniedException e) { // Namespace may have been deleted or permission denied } } } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } return tables.build(); @@ -362,7 +353,7 @@ public void dropTable(ConnectorSession session, SchemaTableName schemaTableName) try { deleteTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } try { @@ -379,7 +370,7 @@ public void dropTable(ConnectorSession session, SchemaTableName schemaTableName) @Override public void dropCorruptedTable(ConnectorSession session, SchemaTableName schemaTableName) { - com.amazonaws.services.glue.model.Table table = dropTableFromMetastore(session, schemaTableName); + software.amazon.awssdk.services.glue.model.Table table = dropTableFromMetastore(session, schemaTableName); String metadataLocation = getTableParameters(table).get(METADATA_LOCATION_PROP); if (metadataLocation == null) { throw new TrinoException(ICEBERG_INVALID_METADATA, format("Table %s is missing [%s] property", schemaTableName, METADATA_LOCATION_PROP)); @@ -423,9 +414,9 @@ public void unregisterTable(ConnectorSession session, SchemaTableName schemaTabl dropTableFromMetastore(session, schemaTableName); } - private com.amazonaws.services.glue.model.Table dropTableFromMetastore(ConnectorSession session, SchemaTableName schemaTableName) + private software.amazon.awssdk.services.glue.model.Table dropTableFromMetastore(ConnectorSession session, SchemaTableName schemaTableName) { - com.amazonaws.services.glue.model.Table table = getTable(session, schemaTableName) + software.amazon.awssdk.services.glue.model.Table table = getTable(session, schemaTableName) .orElseThrow(() -> new TableNotFoundException(schemaTableName)); if (!isIcebergTable(getTableParameters(table))) { throw new UnknownTableTypeException(schemaTableName); @@ -434,7 +425,7 @@ private com.amazonaws.services.glue.model.Table dropTableFromMetastore(Connector try { deleteTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } return table; @@ -445,13 +436,15 @@ public void renameTable(ConnectorSession session, SchemaTableName from, SchemaTa { boolean newTableCreated = false; try { - com.amazonaws.services.glue.model.Table table = getTable(session, from) + software.amazon.awssdk.services.glue.model.Table table = getTable(session, from) .orElseThrow(() -> new TableNotFoundException(from)); - TableInput tableInput = getTableInput(to.getTableName(), Optional.ofNullable(table.getOwner()), getTableParameters(table)); - CreateTableRequest createTableRequest = new CreateTableRequest() - .withDatabaseName(to.getSchemaName()) - .withTableInput(tableInput); - stats.getCreateTable().call(() -> glueClient.createTable(createTableRequest)); + TableInput tableInput = getTableInput(to.getTableName(), Optional.ofNullable(table.owner()), getTableParameters(table)); + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(to.getSchemaName()) + .tableInput(tableInput) + .build(); + stats.getCreateTable().call(() -> + glueClient.createTable(createTableRequest)); newTableCreated = true; deleteTable(from.getSchemaName(), from.getTableName()); } @@ -470,15 +463,14 @@ public void renameTable(ConnectorSession session, SchemaTableName from, SchemaTa } } - private Optional getTable(ConnectorSession session, SchemaTableName schemaTableName) + private Optional getTable(ConnectorSession session, SchemaTableName schemaTableName) { try { - com.amazonaws.services.glue.model.Table table = stats.getGetTable().call(() -> - glueClient.getTable(new GetTableRequest() - .withDatabaseName(schemaTableName.getSchemaName()) - .withName(schemaTableName.getTableName())) - .getTable()); - + GetTableRequest getTableRequest = GetTableRequest.builder() + .databaseName(schemaTableName.getSchemaName()) + .name(schemaTableName.getTableName()).build(); + software.amazon.awssdk.services.glue.model.Table table = stats.getGetTable().call(() -> + glueClient.getTable(getTableRequest).table()); Map parameters = getTableParameters(table); if (isIcebergTable(parameters) && !tableMetadataCache.containsKey(schemaTableName)) { if (viewCache.containsKey(schemaTableName) || materializedViewCache.containsKey(schemaTableName)) { @@ -522,10 +514,10 @@ else if (isPrestoView(parameters) && !viewCache.containsKey(schemaTableName)) { try { TrinoViewUtil.getView(schemaTableName, - Optional.ofNullable(table.getViewOriginalText()), + Optional.ofNullable(table.viewOriginalText()), getTableType(table), parameters, - Optional.ofNullable(table.getOwner())) + Optional.ofNullable(table.owner())) .ifPresent(viewDefinition -> viewCache.put(schemaTableName, viewDefinition)); } catch (RuntimeException e) { @@ -542,30 +534,30 @@ else if (isPrestoView(parameters) && !viewCache.containsKey(schemaTableName)) { private void createTable(String schemaName, TableInput tableInput) { + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(schemaName) + .tableInput(tableInput).build(); stats.getCreateTable().call(() -> - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(schemaName) - .withTableInput(tableInput))); + glueClient.createTable(createTableRequest)); } private void deleteTable(String schema, String table) { + DeleteTableRequest deleteTableRequest = DeleteTableRequest.builder() + .databaseName(schema) + .name(table).build(); stats.getDeleteTable().call(() -> - glueClient.deleteTable(new DeleteTableRequest() - .withDatabaseName(schema) - .withName(table))); + glueClient.deleteTable(deleteTableRequest)); } @Override public String defaultTableLocation(ConnectorSession session, SchemaTableName schemaTableName) { - GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest() - .withName(schemaTableName.getSchemaName()); + GetDatabaseRequest getDatabaseRequest = GetDatabaseRequest.builder() + .name(schemaTableName.getSchemaName()) + .build(); String databaseLocation = stats.getGetDatabase().call(() -> - glueClient.getDatabase(getDatabaseRequest) - .getDatabase() - .getLocationUri()); - + glueClient.getDatabase(getDatabaseRequest).database().locationUri()); String tableName = createNewTableName(schemaTableName.getTableName()); if (databaseLocation == null) { @@ -609,25 +601,26 @@ public void createView(ConnectorSession session, SchemaTableName schemaViewName, private void doCreateView(ConnectorSession session, SchemaTableName schemaViewName, TableInput viewTableInput, boolean replace) { - Optional existing = getTable(session, schemaViewName); + Optional existing = getTable(session, schemaViewName); if (existing.isPresent()) { if (!replace || !isPrestoView(getTableParameters(existing.get()))) { // TODO: ViewAlreadyExists is misleading if the name is used by a table https://github.com/trinodb/trino/issues/10037 throw new ViewAlreadyExistsException(schemaViewName); } - + UpdateTableRequest updateTableRequest = UpdateTableRequest.builder() + .databaseName(schemaViewName.getSchemaName()) + .tableInput(viewTableInput).build(); stats.getUpdateTable().call(() -> - glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(schemaViewName.getSchemaName()) - .withTableInput(viewTableInput))); + glueClient.updateTable(updateTableRequest)); return; } try { + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(schemaViewName.getSchemaName()) + .tableInput(viewTableInput).build(); stats.getCreateTable().call(() -> - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(schemaViewName.getSchemaName()) - .withTableInput(viewTableInput))); + glueClient.createTable(createTableRequest)); } catch (AlreadyExistsException e) { throw new ViewAlreadyExistsException(schemaViewName); @@ -639,18 +632,20 @@ public void renameView(ConnectorSession session, SchemaTableName source, SchemaT { boolean newTableCreated = false; try { - com.amazonaws.services.glue.model.Table existingView = getTable(session, source) + software.amazon.awssdk.services.glue.model.Table existingView = getTable(session, source) .orElseThrow(() -> new TableNotFoundException(source)); viewCache.remove(source); TableInput viewTableInput = getViewTableInput( target.getTableName(), - existingView.getViewOriginalText(), - existingView.getOwner(), + existingView.viewOriginalText(), + existingView.owner(), createViewProperties(session, trinoVersion, TRINO_CREATED_BY_VALUE)); - CreateTableRequest createTableRequest = new CreateTableRequest() - .withDatabaseName(target.getSchemaName()) - .withTableInput(viewTableInput); - stats.getCreateTable().call(() -> glueClient.createTable(createTableRequest)); + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(target.getSchemaName()) + .tableInput(viewTableInput) + .build(); + stats.getCreateTable().call(() -> + glueClient.createTable(createTableRequest)); newTableCreated = true; deleteTable(source.getSchemaName(), source.getTableName()); } @@ -686,7 +681,7 @@ public void dropView(ConnectorSession session, SchemaTableName schemaViewName) viewCache.remove(schemaViewName); deleteTable(schemaViewName.getSchemaName(), schemaViewName.getTableName()); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(HIVE_METASTORE_ERROR, e); } } @@ -699,24 +694,21 @@ public List listViews(ConnectorSession session, Optional namespaces = listNamespaces(session, namespace); for (String glueNamespace : namespaces) { try { - views.addAll(getPaginatedResults( - glueClient::getTables, - new GetTablesRequest().withDatabaseName(glueNamespace), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - stats.getGetTables()) - .map(GetTablesResult::getTableList) - .flatMap(List::stream) - .filter(table -> isPrestoView(getTableParameters(table)) && !isTrinoMaterializedView(getTableType(table), getTableParameters(table))) // TODO isTrinoMaterializedView should not be needed, isPrestoView should not return true for materialized views - .map(table -> new SchemaTableName(glueNamespace, table.getName())) - .collect(toImmutableList())); + GetTablesRequest getTablesRequest = GetTablesRequest.builder().databaseName(glueNamespace).build(); + ImmutableList viewsInNamespace = stats.getGetTables().call(() -> + glueClient.getTablesPaginator(getTablesRequest).stream() + .flatMap(getTablesResponse -> getTablesResponse.tableList().stream()) + .filter(table -> isPrestoView(getTableParameters(table)) && !isTrinoMaterializedView(getTableType(table), getTableParameters(table))) // TODO isTrinoMaterializedView should not be needed, isPrestoView should not return true for materialized views + .map(table -> new SchemaTableName(glueNamespace, table.name())) + .collect(toImmutableList())); + views.addAll(viewsInNamespace); } catch (EntityNotFoundException | AccessDeniedException e) { // Namespace may have been deleted or permission denied } } } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } return views.build(); @@ -735,17 +727,17 @@ public Optional getView(ConnectorSession session, Schem return Optional.empty(); } - Optional table = getTable(session, viewName); + Optional table = getTable(session, viewName); if (table.isEmpty()) { return Optional.empty(); } - com.amazonaws.services.glue.model.Table viewDefinition = table.get(); + software.amazon.awssdk.services.glue.model.Table viewDefinition = table.get(); return TrinoViewUtil.getView( viewName, - Optional.ofNullable(viewDefinition.getViewOriginalText()), + Optional.ofNullable(viewDefinition.viewOriginalText()), getTableType(viewDefinition), getTableParameters(viewDefinition), - Optional.ofNullable(viewDefinition.getOwner())); + Optional.ofNullable(viewDefinition.owner())); } @Override @@ -793,12 +785,13 @@ private void updateView(ConnectorSession session, SchemaTableName viewName, Conn createViewProperties(session, trinoVersion, TRINO_CREATED_BY_VALUE)); try { + UpdateTableRequest updateTableRequest = UpdateTableRequest.builder() + .databaseName(viewName.getSchemaName()) + .tableInput(viewTableInput).build(); stats.getUpdateTable().call(() -> - glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(viewName.getSchemaName()) - .withTableInput(viewTableInput))); + glueClient.updateTable(updateTableRequest)); } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } } @@ -811,24 +804,21 @@ public List listMaterializedViews(ConnectorSession session, Opt List namespaces = listNamespaces(session, namespace); for (String glueNamespace : namespaces) { try { - materializedViews.addAll(getPaginatedResults( - glueClient::getTables, - new GetTablesRequest().withDatabaseName(glueNamespace), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - stats.getGetTables()) - .map(GetTablesResult::getTableList) - .flatMap(List::stream) - .filter(table -> isTrinoMaterializedView(getTableType(table), getTableParameters(table))) - .map(table -> new SchemaTableName(glueNamespace, table.getName())) - .collect(toImmutableList())); + GetTablesRequest getTablesRequest = GetTablesRequest.builder().databaseName(glueNamespace).build(); + ImmutableList materializedViewsInNamespace = stats.getGetTables().call(() -> + glueClient.getTablesPaginator(getTablesRequest).stream() + .flatMap(getTablesResponse -> getTablesResponse.tableList().stream()) + .filter(table -> isTrinoMaterializedView(getTableType(table), getTableParameters(table))) + .map(table -> new SchemaTableName(glueNamespace, table.name())) + .collect(toImmutableList())); + materializedViews.addAll(materializedViewsInNamespace); } catch (EntityNotFoundException | AccessDeniedException e) { // Namespace may have been deleted or permission denied } } } - catch (AmazonServiceException e) { + catch (AwsServiceException e) { throw new TrinoException(ICEBERG_CATALOG_ERROR, e); } return materializedViews.build(); @@ -842,7 +832,7 @@ public void createMaterializedView( boolean replace, boolean ignoreExisting) { - Optional existing = getTable(session, viewName); + Optional existing = getTable(session, viewName); if (existing.isPresent()) { if (!isTrinoMaterializedView(getTableType(existing.get()), getTableParameters(existing.get()))) { @@ -867,10 +857,12 @@ public void createMaterializedView( if (existing.isPresent()) { try { + UpdateTableRequest updateTableRequest = UpdateTableRequest.builder() + .databaseName(viewName.getSchemaName()) + .tableInput(materializedViewTableInput) + .build(); stats.getUpdateTable().call(() -> - glueClient.updateTable(new UpdateTableRequest() - .withDatabaseName(viewName.getSchemaName()) - .withTableInput(materializedViewTableInput))); + glueClient.updateTable(updateTableRequest)); } catch (RuntimeException e) { try { @@ -894,29 +886,29 @@ public void createMaterializedView( @Override public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) { - com.amazonaws.services.glue.model.Table view = getTable(session, viewName) + software.amazon.awssdk.services.glue.model.Table view = getTable(session, viewName) .orElseThrow(() -> new MaterializedViewNotFoundException(viewName)); if (!isTrinoMaterializedView(getTableType(view), getTableParameters(view))) { - throw new TrinoException(UNSUPPORTED_TABLE_TYPE, "Not a Materialized View: " + view.getDatabaseName() + "." + view.getName()); + throw new TrinoException(UNSUPPORTED_TABLE_TYPE, "Not a Materialized View: " + view.databaseName() + "." + view.name()); } materializedViewCache.remove(viewName); dropStorageTable(session, view); - deleteTable(view.getDatabaseName(), view.getName()); + deleteTable(view.databaseName(), view.name()); } - private void dropStorageTable(ConnectorSession session, com.amazonaws.services.glue.model.Table view) + private void dropStorageTable(ConnectorSession session, software.amazon.awssdk.services.glue.model.Table view) { Map parameters = getTableParameters(view); String storageTableName = parameters.get(STORAGE_TABLE); if (storageTableName != null) { String storageSchema = Optional.ofNullable(parameters.get(STORAGE_SCHEMA)) - .orElse(view.getDatabaseName()); + .orElse(view.databaseName()); try { dropTable(session, new SchemaTableName(storageSchema, storageTableName)); } catch (TrinoException e) { - LOG.warn(e, "Failed to drop storage table '%s.%s' for materialized view '%s'", storageSchema, storageTableName, view.getName()); + LOG.warn(e, "Failed to drop storage table '%s.%s' for materialized view '%s'", storageSchema, storageTableName, view.name()); } } } @@ -934,12 +926,12 @@ protected Optional doGetMaterializedView(Co return Optional.empty(); } - Optional maybeTable = getTable(session, viewName); + Optional maybeTable = getTable(session, viewName); if (maybeTable.isEmpty()) { return Optional.empty(); } - com.amazonaws.services.glue.model.Table table = maybeTable.get(); + software.amazon.awssdk.services.glue.model.Table table = maybeTable.get(); if (!isTrinoMaterializedView(getTableType(table), getTableParameters(table))) { return Optional.empty(); } @@ -950,7 +942,7 @@ protected Optional doGetMaterializedView(Co private Optional createMaterializedViewDefinition( ConnectorSession session, SchemaTableName viewName, - com.amazonaws.services.glue.model.Table table) + software.amazon.awssdk.services.glue.model.Table table) { Map materializedViewParameters = getTableParameters(table); String storageTable = materializedViewParameters.get(STORAGE_TABLE); @@ -972,13 +964,13 @@ private Optional createMaterializedViewDefi throw new MaterializedViewMayBeBeingRemovedException(e); } - String viewOriginalText = table.getViewOriginalText(); + String viewOriginalText = table.viewOriginalText(); if (viewOriginalText == null) { throw new TrinoException(ICEBERG_BAD_DATA, "Materialized view did not have original text " + viewName); } return Optional.of(getMaterializedViewDefinition( icebergTable, - Optional.ofNullable(table.getOwner()), + Optional.ofNullable(table.owner()), viewOriginalText, storageTableName)); } @@ -988,18 +980,20 @@ public void renameMaterializedView(ConnectorSession session, SchemaTableName sou { boolean newTableCreated = false; try { - com.amazonaws.services.glue.model.Table glueTable = getTable(session, source) + software.amazon.awssdk.services.glue.model.Table glueTable = getTable(session, source) .orElseThrow(() -> new TableNotFoundException(source)); materializedViewCache.remove(source); Map tableParameters = getTableParameters(glueTable); if (!isTrinoMaterializedView(getTableType(glueTable), tableParameters)) { throw new TrinoException(UNSUPPORTED_TABLE_TYPE, "Not a Materialized View: " + source); } - TableInput tableInput = getMaterializedViewTableInput(target.getTableName(), glueTable.getViewOriginalText(), glueTable.getOwner(), tableParameters); - CreateTableRequest createTableRequest = new CreateTableRequest() - .withDatabaseName(target.getSchemaName()) - .withTableInput(tableInput); - stats.getCreateTable().call(() -> glueClient.createTable(createTableRequest)); + TableInput tableInput = getMaterializedViewTableInput(target.getTableName(), glueTable.viewOriginalText(), glueTable.owner(), tableParameters); + CreateTableRequest createTableRequest = CreateTableRequest.builder() + .databaseName(target.getSchemaName()) + .tableInput(tableInput) + .build(); + stats.getCreateTable().call(() -> + glueClient.createTable(createTableRequest)); newTableCreated = true; deleteTable(source.getSchemaName(), source.getTableName()); } @@ -1037,7 +1031,7 @@ public Optional redirectTable(ConnectorSession session, tableName.getSchemaName(), tableName.getTableName().substring(0, metadataMarkerIndex)); - Optional table = getTable(session, new SchemaTableName(tableNameBase.getSchemaName(), tableNameBase.getTableName())); + Optional table = getTable(session, new SchemaTableName(tableNameBase.getSchemaName(), tableNameBase.getTableName())); if (table.isEmpty() || VIRTUAL_VIEW.name().equals(getTableTypeNullable(table.get()))) { return Optional.empty(); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalogFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalogFactory.java index 1c48d5c44e54..72da5320053c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalogFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalogFactory.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.inject.Inject; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.CatalogName; @@ -28,6 +27,7 @@ import io.trino.spi.type.TypeManager; import org.weakref.jmx.Flatten; import org.weakref.jmx.Managed; +import software.amazon.awssdk.services.glue.GlueClient; import java.util.Optional; @@ -42,7 +42,7 @@ public class TrinoGlueCatalogFactory private final IcebergTableOperationsProvider tableOperationsProvider; private final String trinoVersion; private final Optional defaultSchemaLocation; - private final AWSGlueAsync glueClient; + private final GlueClient glueClient; private final boolean isUniqueTableLocation; private final GlueMetastoreStats stats; @@ -56,7 +56,7 @@ public TrinoGlueCatalogFactory( GlueHiveMetastoreConfig glueConfig, IcebergConfig icebergConfig, GlueMetastoreStats stats, - AWSGlueAsync glueClient) + GlueClient glueClient) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAsyncAdapterProvider.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAdapterProvider.java similarity index 79% rename from plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAsyncAdapterProvider.java rename to plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAdapterProvider.java index e4084971e6eb..a0a3b57169a4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAsyncAdapterProvider.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/AWSGlueAdapterProvider.java @@ -13,9 +13,9 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; +import software.amazon.awssdk.services.glue.GlueClient; -public interface AWSGlueAsyncAdapterProvider +public interface AWSGlueAdapterProvider { - AWSGlueAsync createAWSGlueAsyncAdapter(AWSGlueAsync delegate); + GlueClient createAWSGlueAdapter(GlueClient delegate); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java index ae47487e45d6..146229a9fb3c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java @@ -13,11 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetTableRequest; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.model.DeleteObjectsRequest; @@ -42,14 +37,21 @@ import io.trino.plugin.iceberg.IcebergQueryRunner; import io.trino.plugin.iceberg.SchemaInitializer; import io.trino.testing.QueryRunner; +import io.trino.util.AutoCloseableCloser; import org.apache.iceberg.FileFormat; import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetTableRequest; import java.util.List; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResultsForS3; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableParameters; import static io.trino.plugin.iceberg.IcebergTestUtils.checkParquetFileSorting; import static io.trino.testing.TestingConnectorSession.SESSION; @@ -69,15 +71,14 @@ public class TestIcebergGlueCatalogConnectorSmokeTest { private final String bucketName; private final String schemaName; - private final AWSGlueAsync glueClient; private final TrinoFileSystemFactory fileSystemFactory; + private GlueAsyncClient glueClient; public TestIcebergGlueCatalogConnectorSmokeTest() { super(FileFormat.PARQUET); this.bucketName = requireNonNull(System.getenv("S3_BUCKET"), "Environment S3_BUCKET was not set"); this.schemaName = "test_iceberg_smoke_" + randomNameSuffix(); - glueClient = AWSGlueAsyncClientBuilder.defaultClient(); HdfsConfigurationInitializer initializer = new HdfsConfigurationInitializer(new HdfsConfig(), ImmutableSet.of()); HdfsConfiguration hdfsConfiguration = new DynamicHdfsConfiguration(initializer, ImmutableSet.of()); @@ -104,8 +105,16 @@ protected QueryRunner createQueryRunner() .build(); } + @BeforeClass + public void setup() + throws Exception + { + glueClient = GlueAsyncClient.create(); + } + @AfterClass(alwaysRun = true) public void cleanup() + throws Exception { computeActual("SHOW TABLES").getMaterializedRows() .forEach(table -> getQueryRunner().execute("DROP TABLE " + table.getField(0))); @@ -113,6 +122,10 @@ public void cleanup() // DROP TABLES should clean up any files, but clear the directory manually to be safe deleteDirectory(schemaPath()); + try (AutoCloseableCloser closer = AutoCloseableCloser.create()) { + closer.register(glueClient); + } + glueClient = null; } @Test @@ -146,24 +159,27 @@ public void testRenameSchema() @Override protected void dropTableFromMetastore(String tableName) { - DeleteTableRequest deleteTableRequest = new DeleteTableRequest() - .withDatabaseName(schemaName) - .withName(tableName); - glueClient.deleteTable(deleteTableRequest); - GetTableRequest getTableRequest = new GetTableRequest() - .withDatabaseName(schemaName) - .withName(tableName); - assertThatThrownBy(() -> glueClient.getTable(getTableRequest)) + DeleteTableRequest deleteTableRequest = DeleteTableRequest.builder() + .databaseName(schemaName) + .name(tableName) + .build(); + awsSyncRequest(glueClient::deleteTable, deleteTableRequest, null); + GetTableRequest getTableRequest = GetTableRequest.builder() + .databaseName(schemaName) + .name(tableName) + .build(); + assertThatThrownBy(() -> awsSyncRequest(glueClient::getTable, getTableRequest, null)) .isInstanceOf(EntityNotFoundException.class); } @Override protected String getMetadataLocation(String tableName) { - GetTableRequest getTableRequest = new GetTableRequest() - .withDatabaseName(schemaName) - .withName(tableName); - return getTableParameters(glueClient.getTable(getTableRequest).getTable()) + GetTableRequest getTableRequest = GetTableRequest.builder() + .databaseName(schemaName) + .name(tableName) + .build(); + return getTableParameters(awsSyncRequest(glueClient::getTable, getTableRequest, null).table()) .get("metadata_location"); } @@ -175,7 +191,7 @@ protected void deleteDirectory(String location) ListObjectsV2Request listObjectsRequest = new ListObjectsV2Request() .withBucketName(bucketName) .withPrefix(location); - List keysToDelete = getPaginatedResults( + List keysToDelete = getPaginatedResultsForS3( s3::listObjectsV2, listObjectsRequest, ListObjectsV2Request::setContinuationToken, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java index 95b1d315d7ef..8f4a2fe6c1fd 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java @@ -13,29 +13,27 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.BatchDeleteTableRequest; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.GetTablesRequest; -import com.amazonaws.services.glue.model.GetTablesResult; -import com.amazonaws.services.glue.model.Table; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.trino.plugin.hive.aws.AwsApiCallStats; import io.trino.plugin.iceberg.BaseIcebergMaterializedViewTest; import io.trino.plugin.iceberg.IcebergQueryRunner; import io.trino.plugin.iceberg.SchemaInitializer; import io.trino.testing.QueryRunner; import org.testng.annotations.AfterClass; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.BatchDeleteTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.Table; import java.io.File; import java.nio.file.Files; -import java.util.Collection; import java.util.Set; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncPaginatedRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.testing.TestingNames.randomNameSuffix; public class TestIcebergGlueCatalogMaterializedView @@ -80,21 +78,21 @@ public void cleanup() private static void cleanUpSchema(String schema) { - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); - Set tableNames = getPaginatedResults( - glueClient::getTables, - new GetTablesRequest().withDatabaseName(schema), - GetTablesRequest::setNextToken, - GetTablesResult::getNextToken, - new AwsApiCallStats()) - .map(GetTablesResult::getTableList) - .flatMap(Collection::stream) - .map(Table::getName) - .collect(toImmutableSet()); - glueClient.batchDeleteTable(new BatchDeleteTableRequest() - .withDatabaseName(schema) - .withTablesToDelete(tableNames)); - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(schema)); + GlueAsyncClient glueClient = GlueAsyncClient.create(); + + ImmutableSet.Builder tableNamesBuilder = ImmutableSet.builder(); + awsSyncPaginatedRequest(glueClient.getTablesPaginator(GetTablesRequest.builder().databaseName(schema).build()), + tables -> { + tables.tableList().stream() + .map(Table::name) + .forEach(tableNamesBuilder::add); + }, + new AwsApiCallStats()); + Set tableNames = tableNamesBuilder.build(); + awsSyncRequest(glueClient::batchDeleteTable, BatchDeleteTableRequest.builder() + .databaseName(schema) + .tablesToDelete(tableNames).build(), null); + awsSyncRequest(glueClient::deleteDatabase, DeleteDatabaseRequest.builder() + .name(schema).build(), null); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java index 1ce89a8284d9..16b8bc5a857f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java @@ -13,15 +13,7 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.GetTableVersionsRequest; -import com.amazonaws.services.glue.model.GetTableVersionsResult; -import com.amazonaws.services.glue.model.Table; -import com.amazonaws.services.glue.model.TableInput; -import com.amazonaws.services.glue.model.TableVersion; -import com.amazonaws.services.glue.model.UpdateTableRequest; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.aws.AwsApiCallStats; import io.trino.plugin.iceberg.IcebergQueryRunner; @@ -29,18 +21,26 @@ import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import io.trino.testing.sql.TestTable; +import io.trino.util.AutoCloseableCloser; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTableVersionsRequest; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.TableVersion; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.paginators.GetTableVersionsPublisher; import java.io.File; import java.nio.file.Files; -import java.util.Collection; import java.util.List; import java.util.Optional; -import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncPaginatedRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableParameters; import static io.trino.plugin.iceberg.catalog.glue.GlueIcebergUtil.getTableInput; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -55,13 +55,13 @@ public class TestIcebergGlueCatalogSkipArchive extends AbstractTestQueryFramework { private final String schemaName = "test_iceberg_skip_archive_" + randomNameSuffix(); - private AWSGlueAsync glueClient; + private GlueAsyncClient glueClient; @Override protected QueryRunner createQueryRunner() throws Exception { - glueClient = AWSGlueAsyncClientBuilder.defaultClient(); + glueClient = GlueAsyncClient.create(); File schemaDirectory = Files.createTempDirectory("test_iceberg").toFile(); schemaDirectory.deleteOnExit(); @@ -81,8 +81,13 @@ protected QueryRunner createQueryRunner() @AfterClass(alwaysRun = true) public void cleanup() + throws Exception { assertUpdate("DROP SCHEMA IF EXISTS " + schemaName); + try (AutoCloseableCloser closer = AutoCloseableCloser.create()) { + closer.register(glueClient); + } + glueClient = null; } @Test @@ -91,14 +96,14 @@ public void testSkipArchive() try (TestTable table = new TestTable(getQueryRunner()::execute, "test_skip_archive", "(col int)")) { List tableVersionsBeforeInsert = getTableVersions(schemaName, table.getName()); assertThat(tableVersionsBeforeInsert).hasSize(1); - String versionIdBeforeInsert = getOnlyElement(tableVersionsBeforeInsert).getVersionId(); + String versionIdBeforeInsert = getOnlyElement(tableVersionsBeforeInsert).versionId(); assertUpdate("INSERT INTO " + table.getName() + " VALUES 1", 1); // Verify count of table versions isn't increased, but version id is changed List tableVersionsAfterInsert = getTableVersions(schemaName, table.getName()); assertThat(tableVersionsAfterInsert).hasSize(1); - String versionIdAfterInsert = getOnlyElement(tableVersionsAfterInsert).getVersionId(); + String versionIdAfterInsert = getOnlyElement(tableVersionsAfterInsert).versionId(); assertThat(versionIdBeforeInsert).isNotEqualTo(versionIdAfterInsert); } } @@ -112,9 +117,9 @@ public void testNotRemoveExistingArchive() TableVersion initialVersion = getOnlyElement(tableVersionsBeforeInsert); // Add a new archive using Glue client - Table glueTable = glueClient.getTable(new GetTableRequest().withDatabaseName(schemaName).withName(table.getName())).getTable(); + Table glueTable = awsSyncRequest(glueClient::getTable, GetTableRequest.builder().databaseName(schemaName).name(table.getName()).build(), null).table(); TableInput tableInput = getTableInput(table.getName(), Optional.empty(), getTableParameters(glueTable)); - glueClient.updateTable(new UpdateTableRequest().withDatabaseName(schemaName).withTableInput(tableInput)); + awsSyncRequest(glueClient::updateTable, UpdateTableRequest.builder().databaseName(schemaName).tableInput(tableInput).build(), null); assertThat(getTableVersions(schemaName, table.getName())).hasSize(2); assertUpdate("INSERT INTO " + table.getName() + " VALUES 1", 1); @@ -127,14 +132,12 @@ public void testNotRemoveExistingArchive() private List getTableVersions(String databaseName, String tableName) { - return getPaginatedResults( - glueClient::getTableVersions, - new GetTableVersionsRequest().withDatabaseName(databaseName).withTableName(tableName), - GetTableVersionsRequest::setNextToken, - GetTableVersionsResult::getNextToken, - new AwsApiCallStats()) - .map(GetTableVersionsResult::getTableVersions) - .flatMap(Collection::stream) - .collect(toImmutableList()); + ImmutableList.Builder tableVersionBuilder = ImmutableList.builder(); + GetTableVersionsPublisher tableVersionsPaginator = glueClient.getTableVersionsPaginator( + GetTableVersionsRequest.builder().databaseName(databaseName).tableName(tableName).build()); + awsSyncPaginatedRequest(tableVersionsPaginator, + versions -> tableVersionBuilder.addAll(versions.tableVersions()), + new AwsApiCallStats()); + return tableVersionBuilder.build(); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java index cda4705323cf..ca3d519e949f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java @@ -13,9 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.InvalidInputException; -import com.amazonaws.services.glue.model.OperationTimeoutException; import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.Session; @@ -33,6 +30,9 @@ import io.trino.testing.LocalQueryRunner; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.InvalidInputException; +import software.amazon.awssdk.services.glue.model.OperationTimeoutException; import java.lang.reflect.InvocationTargetException; import java.nio.file.Files; @@ -82,7 +82,7 @@ protected LocalQueryRunner createQueryRunner() .build(); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider = delegate -> newProxy(AWSGlueAsync.class, (proxy, method, methodArgs) -> { + AWSGlueAdapterProvider awsGlueAdapterProvider = delegate -> newProxy(GlueClient.class, (proxy, method, methodArgs) -> { Object result; if (method.getName().equals("createTable")) { throw testException.get(); @@ -102,7 +102,7 @@ protected LocalQueryRunner createQueryRunner() queryRunner.createCatalog( ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAdapterProvider)), Optional.empty(), EMPTY_MODULE), ImmutableMap.of()); dataDirectory = Files.createTempDirectory("test_iceberg_create_table_failure"); @@ -143,7 +143,7 @@ public void testCreateTableFailureMetadataCleanedUp() throws Exception { final String exceptionMessage = "Test-simulated metastore invalid input exception"; - testException.set(new InvalidInputException(exceptionMessage)); + testException.set(InvalidInputException.builder().message(exceptionMessage).build()); testCreateTableFailure(exceptionMessage, false); } @@ -152,7 +152,7 @@ public void testCreateTableFailureMetadataNotCleanedUp() throws Exception { final String exceptionMessage = "Test-simulated metastore operation timeout exception"; - testException.set(new OperationTimeoutException(exceptionMessage)); + testException.set(OperationTimeoutException.builder().message(exceptionMessage).build()); testCreateTableFailure(exceptionMessage, true); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java index 27e320f1bbb8..ee4e5a1a452e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.Session; @@ -28,6 +27,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueClient; import java.lang.reflect.InvocationTargetException; import java.nio.file.Files; @@ -68,7 +68,7 @@ protected LocalQueryRunner createQueryRunner() .build(); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider = delegate -> newProxy(AWSGlueAsync.class, (proxy, method, methodArgs) -> { + AWSGlueAdapterProvider awsGlueAdapterProvider = delegate -> newProxy(GlueClient.class, (proxy, method, methodArgs) -> { Object result; try { result = method.invoke(delegate, methodArgs); @@ -88,7 +88,7 @@ protected LocalQueryRunner createQueryRunner() queryRunner.createCatalog( ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAdapterProvider)), Optional.empty(), EMPTY_MODULE), ImmutableMap.of()); Path dataDirectory = Files.createTempDirectory("iceberg_data"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java index 0385dd644bd5..15b213db24be 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java @@ -13,11 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.CreateDatabaseRequest; -import com.amazonaws.services.glue.model.DatabaseInput; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.filesystem.TrinoFileSystemFactory; @@ -36,6 +31,11 @@ import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.type.TestingTypeManager; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; import java.io.File; import java.io.IOException; @@ -45,6 +45,7 @@ import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.testing.TestingConnectorSession.SESSION; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -61,7 +62,7 @@ public class TestTrinoGlueCatalog protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) { TrinoFileSystemFactory fileSystemFactory = HDFS_FILE_SYSTEM_FACTORY; - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); + GlueClient glueClient = GlueClient.create(); return new TrinoGlueCatalog( new CatalogName("catalog_name"), fileSystemFactory, @@ -87,11 +88,11 @@ public void testNonLowercaseGlueDatabase() // Trino schema names are always lowercase (until https://github.com/trinodb/trino/issues/17) String trinoSchemaName = databaseName.toLowerCase(ENGLISH); - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); - glueClient.createDatabase(new CreateDatabaseRequest() - .withDatabaseInput(new DatabaseInput() - // Currently this is actually stored in lowercase - .withName(databaseName))); + GlueAsyncClient glueClient = GlueAsyncClient.create(); + awsSyncRequest(glueClient::createDatabase, CreateDatabaseRequest.builder() + .databaseInput(DatabaseInput.builder() + // Currently this is actually stored in lowercase + .name(databaseName).build()).build(), null); try { TrinoCatalog catalog = createTrinoCatalog(false); assertThat(catalog.namespaceExists(SESSION, databaseName)).as("catalog.namespaceExists(databaseName)") @@ -122,8 +123,9 @@ public void testNonLowercaseGlueDatabase() .contains(trinoSchemaName); } finally { - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(databaseName)); + awsSyncRequest(glueClient::deleteDatabase, DeleteDatabaseRequest.builder() + .name(databaseName) + .build(), null); } } @@ -135,7 +137,7 @@ public void testDefaultLocation() tmpDirectory.toFile().deleteOnExit(); TrinoFileSystemFactory fileSystemFactory = HDFS_FILE_SYSTEM_FACTORY; - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); + GlueClient glueClient = GlueClient.create(); TrinoCatalog catalogWithDefaultLocation = new TrinoGlueCatalog( new CatalogName("catalog_name"), fileSystemFactory, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingGlueIcebergTableOperationsProvider.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingGlueIcebergTableOperationsProvider.java index 9469f6a0d369..ea910dab0c02 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingGlueIcebergTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingGlueIcebergTableOperationsProvider.java @@ -13,8 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.services.glue.AWSGlueAsync; import com.google.inject.Inject; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.metastore.glue.GlueHiveMetastoreConfig; @@ -24,17 +22,19 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; import io.trino.spi.connector.ConnectorSession; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.services.glue.GlueClient; import java.util.Optional; -import static io.trino.plugin.hive.metastore.glue.GlueClientUtil.createAsyncGlueClient; +import static io.trino.plugin.hive.metastore.glue.GlueClientUtil.createSyncGlueClient; import static java.util.Objects.requireNonNull; public class TestingGlueIcebergTableOperationsProvider implements IcebergTableOperationsProvider { private final TrinoFileSystemFactory fileSystemFactory; - private final AWSGlueAsync glueClient; + private final GlueClient glueClient; private final GlueMetastoreStats stats; @Inject @@ -42,16 +42,16 @@ public TestingGlueIcebergTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, GlueMetastoreStats stats, GlueHiveMetastoreConfig glueConfig, - AWSCredentialsProvider credentialsProvider, - AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider) + AwsCredentialsProvider credentialsProvider, + AWSGlueAdapterProvider awsGlueAdapterProvider) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.stats = requireNonNull(stats, "stats is null"); requireNonNull(glueConfig, "glueConfig is null"); requireNonNull(credentialsProvider, "credentialsProvider is null"); - requireNonNull(awsGlueAsyncAdapterProvider, "awsGlueAsyncAdapterProvider is null"); - this.glueClient = awsGlueAsyncAdapterProvider.createAWSGlueAsyncAdapter( - createAsyncGlueClient(glueConfig, credentialsProvider, Optional.empty(), stats.newRequestMetricsCollector())); + requireNonNull(awsGlueAdapterProvider, "awsGlueAsyncAdapterProvider is null"); + this.glueClient = awsGlueAdapterProvider.createAWSGlueAdapter( + createSyncGlueClient(glueConfig, credentialsProvider, Optional.empty(), stats.newRequestMetricsPublisher())); } @Override diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingIcebergGlueCatalogModule.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingIcebergGlueCatalogModule.java index 8a49035e683b..2374721e5525 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingIcebergGlueCatalogModule.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestingIcebergGlueCatalogModule.java @@ -13,9 +13,6 @@ */ package io.trino.plugin.iceberg.catalog.glue; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.glue.model.Table; import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Provides; @@ -31,6 +28,9 @@ import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.glue.model.Table; import java.util.function.Predicate; @@ -42,11 +42,11 @@ public class TestingIcebergGlueCatalogModule extends AbstractConfigurationAwareModule { - private final AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider; + private final AWSGlueAdapterProvider awsGlueAdapterProvider; - public TestingIcebergGlueCatalogModule(AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider) + public TestingIcebergGlueCatalogModule(AWSGlueAdapterProvider awsGlueAdapterProvider) { - this.awsGlueAsyncAdapterProvider = requireNonNull(awsGlueAsyncAdapterProvider, "awsGlueAsyncAdapterProvider is null"); + this.awsGlueAdapterProvider = requireNonNull(awsGlueAdapterProvider, "awsGlueAsyncAdapterProvider is null"); } @Override @@ -56,11 +56,11 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(IcebergGlueCatalogConfig.class); binder.bind(GlueMetastoreStats.class).in(Scopes.SINGLETON); newExporter(binder).export(GlueMetastoreStats.class).withGeneratedName(); - binder.bind(AWSCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); + binder.bind(AwsCredentialsProvider.class).toProvider(GlueCredentialsProvider.class).in(Scopes.SINGLETON); binder.bind(IcebergTableOperationsProvider.class).to(TestingGlueIcebergTableOperationsProvider.class).in(Scopes.SINGLETON); binder.bind(TrinoCatalogFactory.class).to(TrinoGlueCatalogFactory.class).in(Scopes.SINGLETON); newExporter(binder).export(TrinoCatalogFactory.class).withGeneratedName(); - binder.bind(AWSGlueAsyncAdapterProvider.class).toInstance(awsGlueAsyncAdapterProvider); + binder.bind(AWSGlueAdapterProvider.class).toInstance(awsGlueAdapterProvider); // Required to inject HiveMetastoreFactory for migrate procedure binder.bind(Key.get(boolean.class, HideDeltaLakeTables.class)).toInstance(false); @@ -72,7 +72,7 @@ protected void setup(Binder binder) @Provides @Singleton @ForGlueHiveMetastore - public static RequestHandler2 createRequestHandler(IcebergGlueCatalogConfig config) + public static ExecutionInterceptor createRequestHandler(IcebergGlueCatalogConfig config) { return new SkipArchiveRequestHandler(config.isSkipArchive()); } diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index ed5718db7491..684b363bc9ef 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -23,11 +23,6 @@ aws-java-sdk-core - - com.amazonaws - aws-java-sdk-glue - - com.amazonaws aws-java-sdk-s3 @@ -228,6 +223,21 @@ testng + + software.amazon.awssdk + glue + + + + software.amazon.awssdk + sdk-core + + + + software.amazon.awssdk + utils + + com.clickhouse clickhouse-jdbc @@ -338,6 +348,18 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + + software.amazon.awssdk:utils + + + org.apache.maven.plugins maven-shade-plugin diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java index f728dcdb8c45..db46093fab1f 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java @@ -13,16 +13,15 @@ */ package io.trino.tests.product.deltalake; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.GetTableRequest; -import com.amazonaws.services.glue.model.Table; import com.google.common.collect.ImmutableSet; import io.airlift.log.Logger; import io.trino.tempto.ProductTest; import io.trino.tempto.query.QueryResult; import io.trino.testng.services.Flaky; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.Table; import java.time.Instant; import java.time.temporal.ChronoUnit; @@ -31,6 +30,7 @@ import java.util.stream.Collectors; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.getTableType; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; @@ -54,7 +54,7 @@ public class TestDatabricksWithGlueMetastoreCleanUp @Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH) public void testCleanUpOldTablesUsingDelta() { - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.standard().build(); + GlueAsyncClient glueClient = GlueAsyncClient.create(); long startTime = currentTimeMillis(); List schemas = onTrino().executeQuery("SELECT DISTINCT(table_schema) FROM information_schema.tables") .rows().stream() @@ -67,7 +67,7 @@ public void testCleanUpOldTablesUsingDelta() schemas.forEach(schema -> cleanSchema(schema, startTime, glueClient)); } - private void cleanSchema(String schema, long startTime, AWSGlueAsync glueClient) + private void cleanSchema(String schema, long startTime, GlueAsyncClient glueClient) { Set allTestTableNames = findAllTablesInSchema(schema).stream() .filter(name -> name.toLowerCase(ENGLISH).startsWith("test")) @@ -76,8 +76,8 @@ private void cleanSchema(String schema, long startTime, AWSGlueAsync glueClient) int droppedTablesCount = 0; for (String tableName : allTestTableNames) { try { - Table table = glueClient.getTable(new GetTableRequest().withDatabaseName(schema).withName(tableName)).getTable(); - Instant createTime = table.getCreateTime().toInstant(); + Table table = awsSyncRequest(glueClient::getTable, GetTableRequest.builder().databaseName(schema).name(tableName).build(), null).table(); + Instant createTime = table.createTime(); if (createTime.isBefore(SCHEMA_CLEANUP_THRESHOLD)) { if (getTableType(table).contains("VIEW")) { onTrino().executeQuery(format("DROP VIEW IF EXISTS %s.%s", schema, tableName)); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCleanUpGlueMetastore.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCleanUpGlueMetastore.java index b08200cd6ae1..4e1e1c2cdcc3 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCleanUpGlueMetastore.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeDatabricksCleanUpGlueMetastore.java @@ -13,22 +13,21 @@ */ package io.trino.tests.product.deltalake; -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.Database; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetDatabasesResult; +import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; import io.trino.plugin.hive.aws.AwsApiCallStats; import io.trino.tempto.ProductTest; import org.testng.annotations.Test; +import software.amazon.awssdk.services.glue.GlueAsyncClient; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; import java.util.List; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncPaginatedRequest; +import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.awsSyncRequest; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static java.lang.System.currentTimeMillis; @@ -43,27 +42,24 @@ public class TestDeltaLakeDatabricksCleanUpGlueMetastore @Test(groups = {DELTA_LAKE_DATABRICKS, PROFILE_SPECIFIC_TESTS}) public void testCleanupOrphanedDatabases() { - AWSGlueAsync glueClient = AWSGlueAsyncClientBuilder.defaultClient(); + GlueAsyncClient glueClient = GlueAsyncClient.create(); long creationTimeMillisThreshold = currentTimeMillis() - DAYS.toMillis(1); - List orphanedDatabases = getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - new AwsApiCallStats()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .filter(database -> isOrphanedTestDatabase(database, creationTimeMillisThreshold)) - .map(Database::getName) - .collect(toImmutableList()); + ImmutableList.Builder databaseNames = ImmutableList.builder(); + awsSyncPaginatedRequest(glueClient.getDatabasesPaginator(GetDatabasesRequest.builder().build()), + getDatabasesResponse -> getDatabasesResponse.databaseList() + .stream() + .filter(database -> isOrphanedTestDatabase(database, creationTimeMillisThreshold)) + .forEach(glueDatabase -> databaseNames.add(glueDatabase.name())), + new AwsApiCallStats()); + List orphanedDatabases = databaseNames.build(); if (!orphanedDatabases.isEmpty()) { log.info("Found %s %s* databases that look orphaned, removing", orphanedDatabases.size(), TEST_DATABASE_NAME_PREFIX); orphanedDatabases.forEach(database -> { try { log.info("Deleting %s database", database); - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(database)); + awsSyncRequest(glueClient::deleteDatabase, + DeleteDatabaseRequest.builder().name(database).build(), null); } catch (EntityNotFoundException e) { log.info("Database [%s] not found, could be removed by other cleanup process", database); @@ -77,7 +73,7 @@ public void testCleanupOrphanedDatabases() private static boolean isOrphanedTestDatabase(Database database, long creationTimeMillisThreshold) { - return database.getName().startsWith(TEST_DATABASE_NAME_PREFIX) && - database.getCreateTime().getTime() <= creationTimeMillisThreshold; + return database.name().startsWith(TEST_DATABASE_NAME_PREFIX) && + database.createTime().toEpochMilli() <= creationTimeMillisThreshold; } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/util/DeltaLakeTestUtils.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/util/DeltaLakeTestUtils.java index 4d84b362acf7..e6bfa6dd68ba 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/util/DeltaLakeTestUtils.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/util/DeltaLakeTestUtils.java @@ -13,7 +13,6 @@ */ package io.trino.tests.product.deltalake.util; -import com.amazonaws.services.glue.model.ConcurrentModificationException; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.ObjectListing; @@ -24,6 +23,7 @@ import io.airlift.log.Logger; import io.trino.tempto.query.QueryResult; import org.intellij.lang.annotations.Language; +import software.amazon.awssdk.services.glue.model.ConcurrentModificationException; import java.time.temporal.ChronoUnit; import java.util.List;