From 0f87ddde76331d114c16e5f7df26d69f1c0ec9cb Mon Sep 17 00:00:00 2001 From: Arkadiusz Czajkowski Date: Tue, 1 Mar 2022 16:14:52 +0100 Subject: [PATCH] Add S3 Proxy configuration support --- docs/src/main/sphinx/connector/hive-s3.rst | 16 +++ .../io/trino/plugin/hive/s3/HiveS3Config.java | 97 +++++++++++++++++++ .../s3/TrinoS3ConfigurationInitializer.java | 42 ++++++++ .../plugin/hive/s3/TrinoS3FileSystem.java | 32 ++++++ .../trino/plugin/hive/s3/TrinoS3Protocol.java | 40 ++++++++ .../plugin/hive/s3/TestHiveS3Config.java | 26 ++++- .../plugin/hive/s3/TestTrinoS3FileSystem.java | 85 ++++++++++++++++ 7 files changed, 336 insertions(+), 2 deletions(-) create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3Protocol.java diff --git a/docs/src/main/sphinx/connector/hive-s3.rst b/docs/src/main/sphinx/connector/hive-s3.rst index ddcf4fa348d6..68bb7a8e47a1 100644 --- a/docs/src/main/sphinx/connector/hive-s3.rst +++ b/docs/src/main/sphinx/connector/hive-s3.rst @@ -92,6 +92,22 @@ Property Name Description without staging file to be created in the local file system. ``hive.s3.streaming.part-size`` The part size for S3 streaming upload. Defaults to ``16MB``. + +``hive.s3.proxy.host`` Proxy host to use if connecting through a proxy + +``hive.s3.proxy.port`` Proxy port to use if connecting through a proxy + +``hive.s3.proxy.protocol`` Proxy protocol. HTTP or HTTPS , defaults to ``HTTPS``. + +``hive.s3.proxy.non-proxy-hosts`` Hosts list to access without going through the proxy. + +``hive.s3.proxy.username`` Proxy user name to use if connecting through a proxy + +``hive.s3.proxy.password`` Proxy password name to use if connecting through a proxy + +``hive.s3.proxy.preemptive-basic-auth`` Whether to attempt to authenticate preemptively against proxy + when using base authorization, defaults to ``false``. + ============================================ ================================================================= .. _hive-s3-credentials: diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/HiveS3Config.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/HiveS3Config.java index 6a3e316a316d..aea5e007fedf 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/HiveS3Config.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/HiveS3Config.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.s3; import com.google.common.base.StandardSystemProperty; +import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.ConfigSecuritySensitive; @@ -29,9 +30,12 @@ import javax.validation.constraints.NotNull; import java.io.File; +import java.util.List; import java.util.concurrent.TimeUnit; import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; @DefunctConfig("hive.s3.use-instance-credentials") public class HiveS3Config @@ -68,6 +72,13 @@ public class HiveS3Config private boolean requesterPaysEnabled; private boolean s3StreamingUploadEnabled = true; private DataSize s3StreamingPartSize = DataSize.of(16, MEGABYTE); + private String s3proxyHost; + private Integer s3proxyPort = -1; + private TrinoS3Protocol s3ProxyProtocol = TrinoS3Protocol.HTTPS; + private List s3nonProxyHosts = ImmutableList.of(); + private String s3proxyUsername; + private String s3proxyPassword; + private boolean s3preemptiveBasicProxyAuth; public String getS3AwsAccessKey() { @@ -493,4 +504,90 @@ public HiveS3Config setS3StreamingPartSize(DataSize s3StreamingPartSize) this.s3StreamingPartSize = s3StreamingPartSize; return this; } + + public String getS3ProxyHost() + { + return s3proxyHost; + } + + @Config("hive.s3.proxy.host") + public HiveS3Config setS3ProxyHost(String s3proxyHost) + { + this.s3proxyHost = s3proxyHost; + return this; + } + + public int getS3ProxyPort() + { + return s3proxyPort; + } + + @Config("hive.s3.proxy.port") + public HiveS3Config setS3ProxyPort(int s3proxyPort) + { + this.s3proxyPort = s3proxyPort; + return this; + } + + public TrinoS3Protocol getS3ProxyProtocol() + { + return s3ProxyProtocol; + } + + @Config("hive.s3.proxy.protocol") + public HiveS3Config setS3ProxyProtocol(String s3ProxyProtocol) + { + this.s3ProxyProtocol = TrinoS3Protocol.valueOf( + requireNonNull(s3ProxyProtocol, "s3ProxyProtocol is null") + .toUpperCase(ENGLISH)); + return this; + } + + public List getS3NonProxyHosts() + { + return s3nonProxyHosts; + } + + @Config("hive.s3.proxy.non-proxy-hosts") + public HiveS3Config setS3NonProxyHosts(List s3nonProxyHosts) + { + this.s3nonProxyHosts = ImmutableList.copyOf(s3nonProxyHosts); + return this; + } + + public String getS3ProxyUsername() + { + return s3proxyUsername; + } + + @Config("hive.s3.proxy.username") + public HiveS3Config setS3ProxyUsername(String s3proxyUsername) + { + this.s3proxyUsername = s3proxyUsername; + return this; + } + + public String getS3ProxyPassword() + { + return s3proxyPassword; + } + + @Config("hive.s3.proxy.password") + public HiveS3Config setS3ProxyPassword(String s3proxyPassword) + { + this.s3proxyPassword = s3proxyPassword; + return this; + } + + public boolean getS3PreemptiveBasicProxyAuth() + { + return s3preemptiveBasicProxyAuth; + } + + @Config("hive.s3.proxy.preemptive-basic-auth") + public HiveS3Config setS3PreemptiveBasicProxyAuth(boolean s3preemptiveBasicProxyAuth) + { + this.s3preemptiveBasicProxyAuth = s3preemptiveBasicProxyAuth; + return this; + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3ConfigurationInitializer.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3ConfigurationInitializer.java index 329361c76d31..49b3554ba0f9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3ConfigurationInitializer.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3ConfigurationInitializer.java @@ -21,6 +21,7 @@ import javax.inject.Inject; import java.io.File; +import java.util.List; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_ACCESS_KEY; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_ACL_TYPE; @@ -37,8 +38,15 @@ import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_MAX_RETRY_TIME; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_MULTIPART_MIN_FILE_SIZE; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_MULTIPART_MIN_PART_SIZE; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_NON_PROXY_HOSTS; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PATH_STYLE_ACCESS; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PIN_CLIENT_TO_CURRENT_REGION; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PREEMPTIVE_BASIC_PROXY_AUTH; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PROXY_HOST; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PROXY_PASSWORD; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PROXY_PORT; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PROXY_PROTOCOL; +import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_PROXY_USERNAME; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_REQUESTER_PAYS_ENABLED; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_SECRET_KEY; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_SIGNER_CLASS; @@ -54,6 +62,7 @@ import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_STREAMING_UPLOAD_ENABLED; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_STREAMING_UPLOAD_PART_SIZE; import static io.trino.plugin.hive.s3.TrinoS3FileSystem.S3_USER_AGENT_PREFIX; +import static java.util.stream.Collectors.joining; public class TrinoS3ConfigurationInitializer implements ConfigurationInitializer @@ -90,6 +99,13 @@ public class TrinoS3ConfigurationInitializer private final boolean skipGlacierObjects; private final boolean s3StreamingUploadEnabled; private final DataSize streamingPartSize; + private final String s3proxyHost; + private final int s3proxyPort; + private final TrinoS3Protocol s3ProxyProtocol; + private final List s3nonProxyHosts; + private final String s3proxyUsername; + private final String s3proxyPassword; + private final boolean s3preemptiveBasicProxyAuth; @Inject public TrinoS3ConfigurationInitializer(HiveS3Config config) @@ -126,6 +142,13 @@ public TrinoS3ConfigurationInitializer(HiveS3Config config) this.requesterPaysEnabled = config.isRequesterPaysEnabled(); this.s3StreamingUploadEnabled = config.isS3StreamingUploadEnabled(); this.streamingPartSize = config.getS3StreamingPartSize(); + this.s3proxyHost = config.getS3ProxyHost(); + this.s3proxyPort = config.getS3ProxyPort(); + this.s3ProxyProtocol = config.getS3ProxyProtocol(); + this.s3nonProxyHosts = config.getS3NonProxyHosts(); + this.s3proxyUsername = config.getS3ProxyUsername(); + this.s3proxyPassword = config.getS3ProxyPassword(); + this.s3preemptiveBasicProxyAuth = config.getS3PreemptiveBasicProxyAuth(); } @Override @@ -188,5 +211,24 @@ public void initializeConfiguration(Configuration config) config.setBoolean(S3_REQUESTER_PAYS_ENABLED, requesterPaysEnabled); config.setBoolean(S3_STREAMING_UPLOAD_ENABLED, s3StreamingUploadEnabled); config.setLong(S3_STREAMING_UPLOAD_PART_SIZE, streamingPartSize.toBytes()); + if (s3proxyHost != null) { + config.set(S3_PROXY_HOST, s3proxyHost); + } + if (s3proxyPort > -1) { + config.setInt(S3_PROXY_PORT, s3proxyPort); + } + if (s3ProxyProtocol != null) { + config.set(S3_PROXY_PROTOCOL, s3ProxyProtocol.name()); + } + if (s3nonProxyHosts != null) { + config.set(S3_NON_PROXY_HOSTS, s3nonProxyHosts.stream().collect(joining("|"))); + } + if (s3proxyUsername != null) { + config.set(S3_PROXY_USERNAME, s3proxyUsername); + } + if (s3proxyPassword != null) { + config.set(S3_PROXY_PASSWORD, s3proxyPassword); + } + config.setBoolean(S3_PREEMPTIVE_BASIC_PROXY_AUTH, s3preemptiveBasicProxyAuth); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java index 1bfd8fb5db45..0e21ade96bb8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java @@ -150,6 +150,7 @@ import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.nio.file.Files.createDirectories; import static java.nio.file.Files.createTempFile; +import static java.util.Objects.nonNull; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; @@ -196,6 +197,13 @@ public class TrinoS3FileSystem public static final String S3_STREAMING_UPLOAD_PART_SIZE = "trino.s3.streaming.part-size"; public static final String S3_STORAGE_CLASS = "trino.s3.storage-class"; public static final String S3_ROLE_SESSION_NAME = "trino.s3.role-session-name"; + public static final String S3_PROXY_HOST = "trino.s3.proxy.host"; + public static final String S3_PROXY_PORT = "trino.s3.proxy.port"; + public static final String S3_PROXY_PROTOCOL = "trino.s3.proxy.protocol"; + public static final String S3_NON_PROXY_HOSTS = "trino.s3.proxy.non-proxy-hosts"; + public static final String S3_PROXY_USERNAME = "trino.s3.proxy.username"; + public static final String S3_PROXY_PASSWORD = "trino.s3.proxy.password"; + public static final String S3_PREEMPTIVE_BASIC_PROXY_AUTH = "trino.s3.proxy.preemptive-basic-auth"; private static final Logger log = Logger.get(TrinoS3FileSystem.class); private static final TrinoS3FileSystemStats STATS = new TrinoS3FileSystemStats(); @@ -294,6 +302,30 @@ public void initialize(URI uri, Configuration conf) .withUserAgentPrefix(userAgentPrefix) .withUserAgentSuffix("Trino"); + String proxyHost = conf.get(S3_PROXY_HOST); + if (nonNull(proxyHost)) { + configuration.setProxyHost(proxyHost); + configuration.setProxyPort(conf.getInt(S3_PROXY_PORT, defaults.getS3ProxyPort())); + String proxyProtocol = conf.get(S3_PROXY_PROTOCOL); + if (proxyProtocol != null) { + configuration.setProxyProtocol(TrinoS3Protocol.valueOf(proxyProtocol).getProtocol()); + } + String nonProxyHosts = conf.get(S3_NON_PROXY_HOSTS); + if (nonProxyHosts != null) { + configuration.setNonProxyHosts(nonProxyHosts); + } + String proxyUsername = conf.get(S3_PROXY_USERNAME); + if (proxyUsername != null) { + configuration.setProxyUsername(proxyUsername); + } + String proxyPassword = conf.get(S3_PROXY_PASSWORD); + if (proxyPassword != null) { + configuration.setProxyPassword(proxyPassword); + } + configuration.setPreemptiveBasicProxyAuth( + conf.getBoolean(S3_PREEMPTIVE_BASIC_PROXY_AUTH, defaults.getS3PreemptiveBasicProxyAuth())); + } + this.credentialsProvider = createAwsCredentialsProvider(uri, conf); this.s3 = createAmazonS3Client(conf, configuration); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3Protocol.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3Protocol.java new file mode 100644 index 000000000000..581e7cd480e5 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3Protocol.java @@ -0,0 +1,40 @@ +/* + * 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.s3; + +import com.amazonaws.Protocol; + +public enum TrinoS3Protocol +{ + HTTP(Protocol.HTTP), + HTTPS(Protocol.HTTPS); + + private final Protocol protocol; + + TrinoS3Protocol(Protocol protocol) + { + this.protocol = protocol; + } + + public Protocol getProtocol() + { + return protocol; + } + + @Override + public String toString() + { + return this.protocol.name(); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3Config.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3Config.java index 6bffe30eb10d..cf07834b57c0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3Config.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3Config.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.s3; import com.google.common.base.StandardSystemProperty; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; @@ -68,7 +69,14 @@ public void testDefaults() .setSkipGlacierObjects(false) .setRequesterPaysEnabled(false) .setS3StreamingUploadEnabled(true) - .setS3StreamingPartSize(DataSize.of(16, Unit.MEGABYTE))); + .setS3StreamingPartSize(DataSize.of(16, Unit.MEGABYTE)) + .setS3ProxyHost(null) + .setS3ProxyPort(-1) + .setS3ProxyProtocol("HTTPS") + .setS3NonProxyHosts(ImmutableList.of()) + .setS3ProxyUsername(null) + .setS3ProxyPassword(null) + .setS3PreemptiveBasicProxyAuth(false)); } @Test @@ -110,6 +118,13 @@ public void testExplicitPropertyMappings() .put("hive.s3.requester-pays.enabled", "true") .put("hive.s3.streaming.enabled", "false") .put("hive.s3.streaming.part-size", "15MB") + .put("hive.s3.proxy.host", "localhost") + .put("hive.s3.proxy.port", "14000") + .put("hive.s3.proxy.protocol", "HTTP") + .put("hive.s3.proxy.non-proxy-hosts", "test,test2,test3") + .put("hive.s3.proxy.username", "test") + .put("hive.s3.proxy.password", "test") + .put("hive.s3.proxy.preemptive-basic-auth", "true") .buildOrThrow(); HiveS3Config expected = new HiveS3Config() @@ -144,7 +159,14 @@ public void testExplicitPropertyMappings() .setSkipGlacierObjects(true) .setRequesterPaysEnabled(true) .setS3StreamingUploadEnabled(false) - .setS3StreamingPartSize(DataSize.of(15, Unit.MEGABYTE)); + .setS3StreamingPartSize(DataSize.of(15, Unit.MEGABYTE)) + .setS3ProxyHost("localhost") + .setS3ProxyPort(14000) + .setS3ProxyProtocol("HTTP") + .setS3NonProxyHosts(ImmutableList.of("test", "test2", "test3")) + .setS3ProxyUsername("test") + .setS3ProxyPassword("test") + .setS3PreemptiveBasicProxyAuth(true); assertFullMapping(properties, expected); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestTrinoS3FileSystem.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestTrinoS3FileSystem.java index 994ac4a7d587..bed5ca80407a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestTrinoS3FileSystem.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestTrinoS3FileSystem.java @@ -15,6 +15,7 @@ import com.amazonaws.AmazonWebServiceClient; import com.amazonaws.ClientConfiguration; +import com.amazonaws.Protocol; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; @@ -38,6 +39,7 @@ import com.amazonaws.services.securitytoken.AWSSecurityTokenService; import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient; import com.google.common.base.VerifyException; +import com.google.common.collect.ImmutableList; import io.trino.plugin.hive.s3.TrinoS3FileSystem.UnrecoverableS3OperationException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -583,6 +585,89 @@ public void testSkipGlacierObjectsEnabled() assertSkipGlacierObjects(false); } + @Test + public void testProxyDefaultsS3ClientConfiguration() + throws Exception + { + HiveS3Config hiveS3Config = new HiveS3Config(); + + TrinoS3ConfigurationInitializer configurationInitializer = new TrinoS3ConfigurationInitializer(hiveS3Config); + Configuration trinoFsConfiguration = new Configuration(false); + configurationInitializer.initializeConfiguration(trinoFsConfiguration); + + try (TrinoS3FileSystem fs = new TrinoS3FileSystem()) { + fs.initialize(new URI("s3n://test-bucket/"), trinoFsConfiguration); + ClientConfiguration config = getFieldValue(fs.getS3Client(), AmazonWebServiceClient.class, "clientConfiguration", ClientConfiguration.class); + assertNull(config.getProxyHost()); + assertEquals(config.getProxyPort(), -1); + assertEquals(config.getProxyProtocol(), Protocol.HTTP); + assertEquals(config.getNonProxyHosts(), System.getProperty("http.nonProxyHosts")); + assertNull(config.getProxyUsername()); + assertNull(config.getProxyPassword()); + assertFalse(config.isPreemptiveBasicProxyAuth()); + } + } + + @Test + public void testOnNoHostProxyDefaultsS3ClientConfiguration() + throws Exception + { + HiveS3Config hiveS3Config = new HiveS3Config(); + hiveS3Config.setS3ProxyHost(null); + hiveS3Config.setS3ProxyPort(40000); + hiveS3Config.setS3ProxyProtocol("https"); + hiveS3Config.setS3NonProxyHosts(ImmutableList.of("firsthost.com", "secondhost.com")); + hiveS3Config.setS3ProxyUsername("dummy_username"); + hiveS3Config.setS3ProxyPassword("dummy_password"); + hiveS3Config.setS3PreemptiveBasicProxyAuth(true); + + TrinoS3ConfigurationInitializer configurationInitializer = new TrinoS3ConfigurationInitializer(hiveS3Config); + Configuration trinoFsConfiguration = new Configuration(false); + configurationInitializer.initializeConfiguration(trinoFsConfiguration); + + try (TrinoS3FileSystem fs = new TrinoS3FileSystem()) { + fs.initialize(new URI("s3n://test-bucket/"), trinoFsConfiguration); + ClientConfiguration config = getFieldValue(fs.getS3Client(), AmazonWebServiceClient.class, "clientConfiguration", ClientConfiguration.class); + assertNull(config.getProxyHost()); + assertEquals(config.getProxyPort(), -1); + assertEquals(config.getProxyProtocol(), Protocol.HTTP); + assertEquals(config.getNonProxyHosts(), System.getProperty("http.nonProxyHosts")); + assertNull(config.getProxyUsername()); + assertNull(config.getProxyPassword()); + assertFalse(config.isPreemptiveBasicProxyAuth()); + } + } + + @Test + public void testExplicitProxyS3ClientConfiguration() + throws Exception + { + HiveS3Config hiveS3Config = new HiveS3Config(); + hiveS3Config.setS3ProxyHost("dummy.com"); + hiveS3Config.setS3ProxyPort(40000); + hiveS3Config.setS3ProxyProtocol("https"); + hiveS3Config.setS3NonProxyHosts(ImmutableList.of("firsthost.com", "secondhost.com")); + hiveS3Config.setS3ProxyUsername("dummy_username"); + hiveS3Config.setS3ProxyPassword("dummy_password"); + hiveS3Config.setS3PreemptiveBasicProxyAuth(true); + + TrinoS3ConfigurationInitializer configurationInitializer = new TrinoS3ConfigurationInitializer(hiveS3Config); + Configuration trinoFsConfiguration = new Configuration(false); + configurationInitializer.initializeConfiguration(trinoFsConfiguration); + + try (TrinoS3FileSystem fs = new TrinoS3FileSystem()) { + fs.initialize(new URI("s3n://test-bucket/"), trinoFsConfiguration); + ClientConfiguration config = getFieldValue(fs.getS3Client(), AmazonWebServiceClient.class, "clientConfiguration", ClientConfiguration.class); + assertEquals(config.getProxyHost(), "dummy.com"); + assertEquals(config.getProxyPort(), 40000); + assertEquals(config.getProxyProtocol(), Protocol.HTTPS); + assertEquals(config.getNonProxyHosts(), "firsthost.com|secondhost.com"); + assertEquals(config.getProxyUsername(), "dummy_username"); + assertEquals(config.getProxyPassword(), "dummy_password"); + assertTrue(config.isPreemptiveBasicProxyAuth()); + } + } + private static void assertSkipGlacierObjects(boolean skipGlacierObjects) throws Exception {