From d4be6932ee7465db03ec3d6f79075072d4a3aff0 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Fri, 3 Jan 2025 21:32:06 +0100 Subject: [PATCH] Allow configuring gcs service endpoint --- .../trino/filesystem/gcs/GcsFileSystemConfig.java | 15 +++++++++++++++ .../trino/filesystem/gcs/GcsStorageFactory.java | 5 +++++ .../filesystem/gcs/TestGcsFileSystemConfig.java | 4 ++++ 3 files changed, 24 insertions(+) diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java index 18cb104e348c..f78d1a51e658 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java @@ -26,6 +26,7 @@ import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; +import java.util.Optional; import java.util.concurrent.TimeUnit; import static com.google.common.base.Preconditions.checkState; @@ -39,6 +40,7 @@ public class GcsFileSystemConfig private int batchSize = 100; private String projectId; + private Optional endpoint = Optional.empty(); private boolean useGcsAccessToken; private String jsonKey; @@ -120,6 +122,19 @@ public GcsFileSystemConfig setProjectId(String projectId) return this; } + public Optional getEndpoint() + { + return endpoint; + } + + @ConfigDescription("Endpoint to use for GCS requests") + @Config("gcs.endpoint") + public GcsFileSystemConfig setEndpoint(Optional endpoint) + { + this.endpoint = endpoint; + return this; + } + public boolean isUseGcsAccessToken() { return useGcsAccessToken; diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java index f57dcb9f32dd..f8ea12c4520a 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java @@ -41,6 +41,7 @@ public class GcsStorageFactory public static final String GCS_OAUTH_KEY = "gcs.oauth"; public static final List DEFAULT_SCOPES = ImmutableList.of("https://www.googleapis.com/auth/cloud-platform"); private final String projectId; + private final Optional endpoint; private final boolean useGcsAccessToken; private final Optional jsonGoogleCredential; private final int maxRetries; @@ -56,6 +57,7 @@ public GcsStorageFactory(GcsFileSystemConfig config) { config.validate(); projectId = config.getProjectId(); + endpoint = config.getEndpoint(); useGcsAccessToken = config.isUseGcsAccessToken(); String jsonKey = config.getJsonKey(); String jsonKeyFilePath = config.getJsonKeyFilePath(); @@ -105,6 +107,9 @@ public Storage create(ConnectorIdentity identity) if (projectId != null) { storageOptionsBuilder.setProjectId(projectId); } + + endpoint.ifPresent(storageOptionsBuilder::setHost); + // Note: without uniform strategy we cannot retry idempotent operations. // The trino-filesystem api does not violate the conditions for idempotency, see https://cloud.google.com/storage/docs/retry-strategy#java for details. return storageOptionsBuilder diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java index 22ab1f7f6212..fd9f5cddfd70 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java @@ -23,6 +23,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Map; +import java.util.Optional; import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; @@ -44,6 +45,7 @@ void testDefaults() .setPageSize(100) .setBatchSize(100) .setProjectId(null) + .setEndpoint(Optional.empty()) .setUseGcsAccessToken(false) .setJsonKey(null) .setJsonKeyFilePath(null) @@ -67,6 +69,7 @@ void testExplicitPropertyMappings() .put("gcs.page-size", "10") .put("gcs.batch-size", "11") .put("gcs.project-id", "project") + .put("gcs.endpoint", "http://custom.dns.org:8000") .put("gcs.use-access-token", "true") .put("gcs.json-key", "{}") .put("gcs.json-key-file-path", jsonKeyFile.toString()) @@ -84,6 +87,7 @@ void testExplicitPropertyMappings() .setPageSize(10) .setBatchSize(11) .setProjectId("project") + .setEndpoint(Optional.of("http://custom.dns.org:8000")) .setUseGcsAccessToken(true) .setJsonKey("{}") .setJsonKeyFilePath(jsonKeyFile.toString())