diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index bbf573dbd1d4..f52ab59813a0 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -205,6 +205,32 @@ io.trino trino-exchange-filesystem test + + + com.google.apis + google-api-services-storage + + + com.google.api-client + google-api-client + + + com.google.auto.value + auto-value-annotations + + + com.google.code.gson + gson + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + diff --git a/plugin/trino-exchange-filesystem/pom.xml b/plugin/trino-exchange-filesystem/pom.xml index 173a41e031f8..e1461d5058d5 100644 --- a/plugin/trino-exchange-filesystem/pom.xml +++ b/plugin/trino-exchange-filesystem/pom.xml @@ -111,6 +111,92 @@ azure-storage-blob-batch + + com.google.api + gax + 2.17.0 + + + com.google.protobuf + protobuf-java + + + io.opencensus + opencensus-api + + + org.threeten + threetenbp + + + + + + com.google.auth + google-auth-library-credentials + 1.6.0 + + + + com.google.auth + google-auth-library-oauth2-http + 1.6.0 + + + commons-logging + commons-logging + + + org.apache.httpcomponents + httpcore + + + + + + com.google.cloud + google-cloud-core + 2.5.6 + + + com.google.protobuf + protobuf-java + + + commons-logging + commons-logging + + + + + + com.google.cloud + google-cloud-storage + 2.5.1 + + + com.google.auto.value + auto-value-annotations + + + com.google.guava + listenablefuture + + + com.google.oauth-client + google-oauth-client + + + com.google.protobuf + protobuf-java + + + org.checkerframework + checker-qual + + + + com.google.code.findbugs jsr305 diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchange.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchange.java index 480942416b96..da16a987571b 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchange.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchange.java @@ -17,8 +17,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.Multimap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import io.trino.spi.exchange.Exchange; import io.trino.spi.exchange.ExchangeContext; import io.trino.spi.exchange.ExchangeSinkHandle; @@ -307,11 +305,7 @@ public ExchangeSourceStatistics getExchangeSourceStatistics(ExchangeSourceHandle @Override public void close() { - ImmutableList.Builder> futures = ImmutableList.builder(); - for (Integer taskPartitionId : allSinks) { - futures.add(exchangeStorage.deleteRecursively(getTaskOutputDirectory(taskPartitionId))); - } - stats.getCloseExchange().record(Futures.allAsList(futures.build())); + stats.getCloseExchange().record(exchangeStorage.deleteRecursively(allSinks.stream().map(this::getTaskOutputDirectory).collect(toImmutableList()))); } private static String generateRandomizedPrefix() diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeModule.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeModule.java index 31ff81f82a7c..36721bed7dc0 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeModule.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeModule.java @@ -29,6 +29,8 @@ import java.util.List; import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.plugin.exchange.filesystem.s3.S3FileSystemExchangeStorage.CompatibilityMode.AWS; +import static io.trino.plugin.exchange.filesystem.s3.S3FileSystemExchangeStorage.CompatibilityMode.GCP; import static io.trino.spi.StandardErrorCode.CONFIGURATION_INVALID; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static java.lang.String.format; @@ -53,11 +55,13 @@ protected void setup(Binder binder) if (scheme == null || scheme.equals("file")) { binder.bind(FileSystemExchangeStorage.class).to(LocalFileSystemExchangeStorage.class).in(Scopes.SINGLETON); } - else if (ImmutableSet.of("s3", "s3a", "s3n").contains(scheme)) { + else if (ImmutableSet.of("s3", "gs").contains(scheme)) { binder.bind(S3FileSystemExchangeStorageStats.class).in(Scopes.SINGLETON); newExporter(binder).export(S3FileSystemExchangeStorageStats.class).withGeneratedName(); binder.bind(FileSystemExchangeStorage.class).to(S3FileSystemExchangeStorage.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(ExchangeS3Config.class); + S3FileSystemExchangeStorage.CompatibilityMode compatibilityMode = scheme.equals("gs") ? GCP : AWS; + binder.bind(S3FileSystemExchangeStorage.CompatibilityMode.class).toInstance(compatibilityMode); } else if (ImmutableSet.of("abfs", "abfss").contains(scheme)) { binder.bind(FileSystemExchangeStorage.class).to(AzureBlobFileSystemExchangeStorage.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeSink.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeSink.java index 100bc756752d..9ac90b0c3aeb 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeSink.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeSink.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.exchange.filesystem; +import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -203,7 +204,7 @@ public synchronized CompletableFuture abort() return stats.getExchangeSinkAbort().record(toCompletableFuture(Futures.transformAsync( abortFuture, - ignored -> exchangeStorage.deleteRecursively(outputDirectory), + ignored -> exchangeStorage.deleteRecursively(ImmutableList.of(outputDirectory)), directExecutor()))); } diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeStorage.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeStorage.java index a974d0bf398d..97ba4fd20aed 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeStorage.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/FileSystemExchangeStorage.java @@ -36,7 +36,7 @@ public interface FileSystemExchangeStorage ListenableFuture createEmptyFile(URI file); - ListenableFuture deleteRecursively(URI dir); + ListenableFuture deleteRecursively(List directories); List listFiles(URI dir) throws IOException; diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/azure/AzureBlobFileSystemExchangeStorage.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/azure/AzureBlobFileSystemExchangeStorage.java index e45471cb0cef..1841cb15e672 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/azure/AzureBlobFileSystemExchangeStorage.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/azure/AzureBlobFileSystemExchangeStorage.java @@ -158,12 +158,14 @@ public ListenableFuture createEmptyFile(URI file) } @Override - public ListenableFuture deleteRecursively(URI dir) + public ListenableFuture deleteRecursively(List directories) { - return asVoid(Futures.transformAsync( - toListenableFuture(listObjectsRecursively(dir).byPage().collectList().toFuture()), - pagedResponseList -> deleteObjects(getContainerName(dir), pagedResponseList), - directExecutor())); + return asVoid(Futures.allAsList(directories.stream() + .map(dir -> Futures.transformAsync( + toListenableFuture(listObjectsRecursively(dir).byPage().collectList().toFuture()), + pagedResponseList -> deleteObjects(getContainerName(dir), pagedResponseList), + directExecutor())) + .collect(toImmutableList()))); } @Override diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java index 68fb8eb8a4b7..870f0573c349 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java @@ -107,13 +107,15 @@ public ListenableFuture createEmptyFile(URI file) } @Override - public ListenableFuture deleteRecursively(URI dir) + public ListenableFuture deleteRecursively(List directories) { - try { - MoreFiles.deleteRecursively(Paths.get(dir.getPath()), ALLOW_INSECURE); - } - catch (IOException | RuntimeException e) { - return immediateFailedFuture(e); + for (URI dir : directories) { + try { + MoreFiles.deleteRecursively(Paths.get(dir.getPath()), ALLOW_INSECURE); + } + catch (IOException | RuntimeException e) { + return immediateFailedFuture(e); + } } return immediateVoidFuture(); } diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java index ba108f8757d7..b680c425732f 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/ExchangeS3Config.java @@ -48,6 +48,7 @@ public class ExchangeS3Config private int asyncClientConcurrency = 100; private int asyncClientMaxPendingConnectionAcquires = 10000; private Duration connectionAcquisitionTimeout = new Duration(1, MINUTES); + private Optional gcsJsonKeyFilePath = Optional.empty(); public String getS3AwsAccessKey() { @@ -205,4 +206,16 @@ public ExchangeS3Config setConnectionAcquisitionTimeout(Duration connectionAcqui this.connectionAcquisitionTimeout = connectionAcquisitionTimeout; return this; } + + public Optional getGcsJsonKeyFilePath() + { + return gcsJsonKeyFilePath; + } + + @Config("exchange.gcs.json-key-file-path") + public ExchangeS3Config setGcsJsonKeyFilePath(String gcsJsonKeyFilePath) + { + this.gcsJsonKeyFilePath = Optional.ofNullable(gcsJsonKeyFilePath); + return this; + } } diff --git a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java index 51dffc1da26a..3f71b86df307 100644 --- a/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java +++ b/plugin/trino-exchange-filesystem/src/main/java/io/trino/plugin/exchange/filesystem/s3/S3FileSystemExchangeStorage.java @@ -13,12 +13,20 @@ */ package io.trino.plugin.exchange.filesystem.s3; +import com.google.api.gax.paging.Page; +import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageBatch; +import com.google.cloud.storage.StorageOptions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.io.Closer; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; import io.airlift.slice.Slice; import io.airlift.slice.SliceInput; import io.airlift.slice.Slices; @@ -76,6 +84,7 @@ import javax.crypto.SecretKey; import javax.inject.Inject; +import java.io.FileInputStream; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -83,6 +92,8 @@ import java.util.List; import java.util.Optional; import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -92,22 +103,29 @@ import static com.google.common.util.concurrent.Futures.immediateFailedFuture; import static com.google.common.util.concurrent.Futures.immediateVoidFuture; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static io.airlift.concurrent.MoreFutures.asVoid; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.concurrent.MoreFutures.toListenableFuture; +import static io.airlift.concurrent.Threads.threadsNamed; import static io.trino.plugin.exchange.filesystem.FileSystemExchangeManager.PATH_SEPARATOR; +import static io.trino.plugin.exchange.filesystem.s3.S3FileSystemExchangeStorage.CompatibilityMode.GCP; import static io.trino.plugin.exchange.filesystem.s3.S3RequestUtil.configureEncryption; import static java.lang.Math.min; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNullElseGet; +import static java.util.concurrent.TimeUnit.SECONDS; import static software.amazon.awssdk.core.client.config.SdkAdvancedClientOption.USER_AGENT_PREFIX; import static software.amazon.awssdk.core.client.config.SdkAdvancedClientOption.USER_AGENT_SUFFIX; public class S3FileSystemExchangeStorage implements FileSystemExchangeStorage { - private static final String DIRECTORY_SUFFIX = "_$folder$"; + public enum CompatibilityMode { + AWS, + GCP + } private final S3FileSystemExchangeStorageStats stats; private final Optional region; @@ -116,9 +134,15 @@ public class S3FileSystemExchangeStorage private final S3Client s3Client; private final S3AsyncClient s3AsyncClient; private final StorageClass storageClass; + private final CompatibilityMode compatibilityMode; + + // GCS specific + private final Optional gcsClient; + private final Optional gcsDeleteExecutor; @Inject - public S3FileSystemExchangeStorage(S3FileSystemExchangeStorageStats stats, ExchangeS3Config config) + public S3FileSystemExchangeStorage(S3FileSystemExchangeStorageStats stats, ExchangeS3Config config, CompatibilityMode compatibilityMode) + throws IOException { this.stats = requireNonNull(stats, "stats is null"); requireNonNull(config, "config is null"); @@ -126,6 +150,7 @@ public S3FileSystemExchangeStorage(S3FileSystemExchangeStorageStats stats, Excha this.endpoint = config.getS3Endpoint(); this.multiUploadPartSize = toIntExact(config.getS3UploadPartSize().toBytes()); this.storageClass = config.getStorageClass(); + this.compatibilityMode = requireNonNull(compatibilityMode, "compatibilityMode is null"); AwsCredentialsProvider credentialsProvider = createAwsCredentialsProvider(config); RetryPolicy retryPolicy = RetryPolicy.builder(config.getRetryMode()) @@ -144,6 +169,29 @@ public S3FileSystemExchangeStorage(S3FileSystemExchangeStorageStats stats, Excha config.getAsyncClientConcurrency(), config.getAsyncClientMaxPendingConnectionAcquires(), config.getConnectionAcquisitionTimeout()); + + if (compatibilityMode == GCP) { + if (config.getGcsJsonKeyFilePath().isPresent()) { + Credentials credentials = GoogleCredentials.fromStream(new FileInputStream(config.getGcsJsonKeyFilePath().get())); + this.gcsClient = Optional.of(StorageOptions.newBuilder().setCredentials(credentials).build().getService()); + } + else { + this.gcsClient = Optional.of(StorageOptions.getDefaultInstance().getService()); + } + ThreadPoolExecutor executor = new ThreadPoolExecutor( + 100, + 100, + 60L, + SECONDS, + new LinkedBlockingQueue<>(), + threadsNamed("gcs-delete-%s")); + executor.allowCoreThreadTimeOut(true); + this.gcsDeleteExecutor = Optional.of(listeningDecorator(executor)); + } + else { + this.gcsClient = Optional.empty(); + this.gcsDeleteExecutor = Optional.empty(); + } } @Override @@ -188,17 +236,33 @@ public ListenableFuture createEmptyFile(URI file) } @Override - public ListenableFuture deleteRecursively(URI dir) + public ListenableFuture deleteRecursively(List directories) { - ImmutableList.Builder keys = ImmutableList.builder(); - return stats.getDeleteRecursively().record(transformFuture(Futures.transformAsync( - toListenableFuture((listObjectsRecursively(dir).subscribe(listObjectsV2Response -> - listObjectsV2Response.contents().stream().map(S3Object::key).forEach(keys::add)))), - ignored -> { - keys.add(keyFromUri(dir) + DIRECTORY_SUFFIX); - return deleteObjects(getBucketName(dir), keys.build()); - }, - directExecutor()))); + if (compatibilityMode == GCP) { + // GCS is not compatible with S3's multi-object delete API https://cloud.google.com/storage/docs/migrating#methods-comparison + Storage storage = gcsClient.orElseThrow(() -> new IllegalStateException("gcsClient is expected to be initialized")); + ListeningExecutorService deleteExecutor = gcsDeleteExecutor.orElseThrow(() -> new IllegalStateException("gcsDeleteExecutor is expected to be initialized")); + return stats.getDeleteRecursively().record(asVoid(deleteExecutor.submit(() -> { + StorageBatch batch = storage.batch(); + for (URI dir : directories) { + Page blobs = storage.list(getBucketName(dir), Storage.BlobListOption.prefix(keyFromUri(dir))); + for (Blob blob : blobs.iterateAll()) { + batch.delete(blob.getBlobId()); + } + } + batch.submit(); + }))); + } + else { + return stats.getDeleteRecursively().record(asVoid(Futures.allAsList(directories.stream().map(dir -> { + ImmutableList.Builder keys = ImmutableList.builder(); + return transformFuture(Futures.transformAsync( + toListenableFuture((listObjectsRecursively(dir).subscribe(listObjectsV2Response -> + listObjectsV2Response.contents().stream().map(S3Object::key).forEach(keys::add)))), + ignored -> deleteObjects(getBucketName(dir), keys.build()), + directExecutor())); + }).collect(toImmutableList())))); + } } @Override @@ -267,6 +331,7 @@ public void close() try (Closer closer = Closer.create()) { closer.register(s3Client::close); closer.register(s3AsyncClient::close); + gcsDeleteExecutor.ifPresent(listeningExecutorService -> closer.register(listeningExecutorService::shutdown)); } } @@ -579,12 +644,11 @@ private void fillBuffer() Optional secretKey = currentFile.getSecretKey(); for (int i = 0; i < readableParts && fileOffset < fileSize; ++i) { int length = (int) min(partSize, fileSize - fileOffset); - int partNumber = (int) (fileOffset / partSize + 1); GetObjectRequest.Builder getObjectRequestBuilder = GetObjectRequest.builder() .key(key) .bucket(bucketName) - .partNumber(partNumber); + .range("bytes=" + fileOffset + "-" + (fileOffset + length - 1)); configureEncryption(secretKey, getObjectRequestBuilder); ListenableFuture getObjectFuture = toListenableFuture(s3AsyncClient.getObject(getObjectRequestBuilder.build(), diff --git a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/containers/MinioStorage.java b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/containers/MinioStorage.java index 942a38a05da1..f81170603336 100644 --- a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/containers/MinioStorage.java +++ b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/containers/MinioStorage.java @@ -86,7 +86,7 @@ public void close() public static Map getExchangeManagerProperties(MinioStorage minioStorage) { return ImmutableMap.builder() - .put("exchange.base-directories", "s3n://" + minioStorage.getBucketName()) + .put("exchange.base-directories", "s3://" + minioStorage.getBucketName()) // TODO: enable exchange encryption after https is supported for Trino MinIO .put("exchange.encryption-enabled", "false") // to trigger file split in some tests diff --git a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java index 227175350c47..3c06c2ed09e3 100644 --- a/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java +++ b/plugin/trino-exchange-filesystem/src/test/java/io/trino/plugin/exchange/filesystem/s3/TestExchangeS3Config.java @@ -45,7 +45,8 @@ public void testDefaults() .setRetryMode(RetryMode.ADAPTIVE) .setAsyncClientConcurrency(100) .setAsyncClientMaxPendingConnectionAcquires(10000) - .setConnectionAcquisitionTimeout(new Duration(1, MINUTES))); + .setConnectionAcquisitionTimeout(new Duration(1, MINUTES)) + .setGcsJsonKeyFilePath(null)); } @Test @@ -64,6 +65,7 @@ public void testExplicitPropertyMappings() .put("exchange.s3.async-client-concurrency", "202") .put("exchange.s3.async-client-max-pending-connection-acquires", "999") .put("exchange.s3.async-client-connection-acquisition-timeout", "5m") + .put("exchange.gcs.json-key-file-path", "/path/to/gcs_keyfile.json") .buildOrThrow(); ExchangeS3Config expected = new ExchangeS3Config() @@ -78,7 +80,8 @@ public void testExplicitPropertyMappings() .setRetryMode(RetryMode.STANDARD) .setAsyncClientConcurrency(202) .setAsyncClientMaxPendingConnectionAcquires(999) - .setConnectionAcquisitionTimeout(new Duration(5, MINUTES)); + .setConnectionAcquisitionTimeout(new Duration(5, MINUTES)) + .setGcsJsonKeyFilePath("/path/to/gcs_keyfile.json"); assertFullMapping(properties, expected); } diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 0aa2a21f8157..0c755625774e 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -323,6 +323,32 @@ io.trino trino-exchange-filesystem test + + + com.google.apis + google-api-services-storage + + + com.google.api-client + google-api-client + + + com.google.auto.value + auto-value-annotations + + + com.google.code.gson + gson + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 330c2861a92f..4f0fe6513277 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -277,6 +277,32 @@ io.trino trino-exchange-filesystem test + + + com.google.apis + google-api-services-storage + + + com.google.api-client + google-api-client + + + com.google.auto.value + auto-value-annotations + + + com.google.code.gson + gson + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + +