You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "akshayakp97 (via GitHub)" <gi...@apache.org> on 2023/05/08 23:35:15 UTC

[GitHub] [iceberg] akshayakp97 opened a new pull request, #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

akshayakp97 opened a new pull request, #7562:
URL: https://github.com/apache/iceberg/pull/7562

   Subtask of #7516 
   
   This PR creates separate class `HttpClientProperties`. Also, move all s3 related methods into `S3FileIOProperties` since we plan to deprecate `AwsProperties`. 
   
   As part of #7156, when `S3FileIOAwsClientFactory` is added for creating S3 client, we will use methods from `S3FileIOProperties` and  `HttpClientProperties`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189102112


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(

Review Comment:
   we should add javadoc explaining this resolution logic



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189103004


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -650,4 +663,71 @@ private Set<Tag> toS3Tags(Map<String, String> properties, String prefix) {
   private boolean keyIdAccessKeyBothConfigured() {
     return (accessKeyId == null) == (secretAccessKey == null);
   }
+
+  public <T extends S3ClientBuilder> void applyCredentialConfigurations(
+      AwsClientProperties awsClientProperties, T builder) {
+    builder.credentialsProvider(
+        isRemoteSigningEnabled
+            ? AnonymousCredentialsProvider.create()
+            : awsClientProperties.credentialsProvider(accessKeyId, secretAccessKey, sessionToken));
+  }
+
+  /**
+   * Configure services settings for an S3 client. The settings include: s3DualStack,
+   * s3UseArnRegion, s3PathStyleAccess, and s3Acceleration
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(s3FileIOProperties::applyS3ServiceConfigurations)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyServiceConfigurations(T builder) {
+    builder
+        .dualstackEnabled(isDualStackEnabled)
+        .serviceConfiguration(
+            S3Configuration.builder()
+                .pathStyleAccessEnabled(isPathStyleAccess)
+                .useArnRegionEnabled(isUseArnRegionEnabled)
+                .accelerateModeEnabled(isAccelerationEnabled)
+                .build());
+  }
+
+  /**
+   * Configure a signer for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(s3FileIOProperties::applyS3SignerConfiguration)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applySignerConfiguration(T builder) {
+    if (isRemoteSigningEnabled) {
+      builder.overrideConfiguration(
+          c ->
+              c.putAdvancedOption(
+                  SdkAdvancedClientOption.SIGNER, S3V4RestSignerClient.create(allProperties)));
+    }
+  }
+
+  /**
+   * Override the endpoint for an S3 client.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyS3EndpointConfigurations)
+   * </pre>
+   */
+  public <T extends S3ClientBuilder> void applyEndpointConfigurations(T builder) {
+    configureEndpoint(builder, endpoint);
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  private <T extends SdkClientBuilder> void configureEndpoint(T builder, String endpoint) {

Review Comment:
   can we just merge this method to `applyEndpointConfigurations`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189150481


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  /**
+   * Returns a credentials provider instance. If params were set, we return a new credentials
+   * instance. If none of the params are set, we try to dynamically load the provided credentials
+   * provider class. If credential provider class wasn't set, we fallback to default credentials
+   * provider.
+   *
+   * @param accessKeyId the AWS access key ID
+   * @param secretAccessKey the AWS secret access key
+   * @param sessionToken the AWS session token
+   * @return a credentials provider instance
+   */
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) {
+      if (Strings.isNullOrEmpty(sessionToken)) {
+        return StaticCredentialsProvider.create(
+            AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+      } else {
+        return StaticCredentialsProvider.create(
+            AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+      }
+    }
+
+    if (!Strings.isNullOrEmpty(this.clientCredentialsProvider)) {
+      return credentialsProvider(this.clientCredentialsProvider);
+    }
+
+    return DefaultCredentialsProvider.create();
+  }
+
+  /**
+   * Tries to first dynamically load the credentials provider class. If successful, try to invoke

Review Comment:
   We don't typically add javadoc for private methods, this can be moved as a part of the public method doc to explain how the credential provider dynamic loading works.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189101498


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {

Review Comment:
   Maybe we should use `Strings.isNullOrEmpty` for all these



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayakp97 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "akshayakp97 (via GitHub)" <gi...@apache.org>.
akshayakp97 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189122726


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {

Review Comment:
   `secretAccessKey` was missing here - https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java#L1695
   ill update it 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189102313


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {
+      if (sessionToken == null) {
+        return StaticCredentialsProvider.create(
+            AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+      } else {
+        return StaticCredentialsProvider.create(
+            AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+      }
+    }
+
+    if (!Strings.isNullOrEmpty(this.clientCredentialsProvider)) {
+      return credentialsProvider(this.clientCredentialsProvider);
+    }
+
+    return DefaultCredentialsProvider.create();
+  }
+
+  private AwsCredentialsProvider credentialsProvider(String credentialsProviderClass) {
+    Class<?> providerClass;
+    try {
+      providerClass = DynClasses.builder().impl(credentialsProviderClass).buildChecked();
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Cannot load class %s, it does not exist in the classpath", credentialsProviderClass),
+          e);
+    }
+
+    Preconditions.checkArgument(
+        AwsCredentialsProvider.class.isAssignableFrom(providerClass),
+        String.format(
+            "Cannot initialize %s, it does not implement %s.",
+            credentialsProviderClass, AwsCredentialsProvider.class.getName()));
+
+    // try to invoke 'create(Map<String, String>)' static method, otherwise fallback to 'create()'

Review Comment:
   once javadoc is added, this can be removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189125137


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {

Review Comment:
   yeah, that's probably a miss we can fix here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayakp97 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "akshayakp97 (via GitHub)" <gi...@apache.org>.
akshayakp97 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189063893


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -493,24 +493,37 @@ public class AwsProperties implements Serializable {
    * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
    * {@link #HTTP_CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see
    * HTTP_CLIENT_TYPE_* defined below.
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE = "http-client.type";
+  @Deprecated public static final String HTTP_CLIENT_TYPE = "http-client.type";
 
   /**
    * If this is set under {@link #HTTP_CLIENT_TYPE}, {@link
    * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
    * Client in {@link AwsClientFactory}
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE_URLCONNECTION = "urlconnection";
+  @Deprecated public static final String HTTP_CLIENT_TYPE_URLCONNECTION = "urlconnection";
 
   /**
    * If this is set under {@link #HTTP_CLIENT_TYPE}, {@link
    * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
    * AwsClientFactory}
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE_APACHE = "apache";
+  @Deprecated public static final String HTTP_CLIENT_TYPE_APACHE = "apache";
 
-  public static final String HTTP_CLIENT_TYPE_DEFAULT = HTTP_CLIENT_TYPE_APACHE;
+  /**
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead

Review Comment:
   yup



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayakp97 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "akshayakp97 (via GitHub)" <gi...@apache.org>.
akshayakp97 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189068595


##########
aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
+
+public class HttpClientProperties {
+
+  /**
+   * The type of {@link software.amazon.awssdk.http.SdkHttpClient} implementation used by {@link
+   * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
+   * {@link #CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see CLIENT_TYPE_*
+   * defined below.
+   */
+  public static final String CLIENT_TYPE = "http-client.type";
+
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
+   * AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_APACHE = "apache";
+
+  private static final String CLIENT_PREFIX = "http-client.";
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
+   * Client in {@link AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection";
+
+  public static final String CLIENT_TYPE_DEFAULT = CLIENT_TYPE_APACHE;
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS =
+      "http-client.urlconnection.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_SOCKET_TIMEOUT_MS =
+      "http-client.urlconnection.socket-timeout-ms";
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIMEOUT_MS =
+      "http-client.apache.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms";
+  /**
+   * Used to configure the connection acquisition timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS =
+      "http-client.apache.connection-acquisition-timeout-ms";
+  /**
+   * Used to configure the connection max idle time in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS =
+      "http-client.apache.connection-max-idle-time-ms";
+  /**
+   * Used to configure the connection time to live in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS =
+      "http-client.apache.connection-time-to-live-ms";
+  /**
+   * Used to configure whether to enable the expect continue setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_EXPECT_CONTINUE_ENABLED =
+      "http-client.apache.expect-continue-enabled";
+  /**
+   * Used to configure the max connections number for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections";
+  /**
+   * Used to configure whether to enable the tcp keep alive setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_TCP_KEEP_ALIVE_ENABLED =
+      "http-client.apache.tcp-keep-alive-enabled";
+  /**
+   * Used to configure whether to use idle connection reaper for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is enabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED =
+      "http-client.apache.use-idle-connection-reaper-enabled";
+
+  private String httpClientType;
+  private final Map<String, String> httpClientProperties;
+
+  public HttpClientProperties() {
+    this.httpClientType = CLIENT_TYPE_DEFAULT;
+    this.httpClientProperties = Collections.emptyMap();
+  }
+
+  public HttpClientProperties(Map<String, String> properties) {
+    this.httpClientType =
+        PropertyUtil.propertyAsString(properties, CLIENT_TYPE, CLIENT_TYPE_DEFAULT);
+    this.httpClientProperties =
+        PropertyUtil.filterProperties(properties, key -> key.startsWith(CLIENT_PREFIX));
+  }
+
+  /**
+   * Configure the httpClient for a client according to the HttpClientType. The two supported
+   * HttpClientTypes are urlconnection and apache
+   *
+   * <p>Sample usage:

Review Comment:
   I believe we don't need to close the paragraph tag`<\p>`? oracle.com/technical-resources/articles/java/javadoc-tool.html#format and https://stackoverflow.com/a/47690366 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1188007551


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -324,6 +341,30 @@ public class S3FileIOProperties implements Serializable {
 
   public static final boolean PRELOAD_CLIENT_ENABLED_DEFAULT = false;
 
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and S3 client factory class will
+   * use this provider to get AWS credentials provided instead of reading the default credential
+   * chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";

Review Comment:
   this should not be in `S3FileIOProperties`, this is generic configuration for all AWS clients, thus the prefix `client.`. So we could create a class `AwsClientProperties` for them



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] akshayakp97 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "akshayakp97 (via GitHub)" <gi...@apache.org>.
akshayakp97 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189068595


##########
aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
+
+public class HttpClientProperties {
+
+  /**
+   * The type of {@link software.amazon.awssdk.http.SdkHttpClient} implementation used by {@link
+   * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
+   * {@link #CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see CLIENT_TYPE_*
+   * defined below.
+   */
+  public static final String CLIENT_TYPE = "http-client.type";
+
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
+   * AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_APACHE = "apache";
+
+  private static final String CLIENT_PREFIX = "http-client.";
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
+   * Client in {@link AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection";
+
+  public static final String CLIENT_TYPE_DEFAULT = CLIENT_TYPE_APACHE;
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS =
+      "http-client.urlconnection.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_SOCKET_TIMEOUT_MS =
+      "http-client.urlconnection.socket-timeout-ms";
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIMEOUT_MS =
+      "http-client.apache.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms";
+  /**
+   * Used to configure the connection acquisition timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS =
+      "http-client.apache.connection-acquisition-timeout-ms";
+  /**
+   * Used to configure the connection max idle time in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS =
+      "http-client.apache.connection-max-idle-time-ms";
+  /**
+   * Used to configure the connection time to live in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS =
+      "http-client.apache.connection-time-to-live-ms";
+  /**
+   * Used to configure whether to enable the expect continue setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_EXPECT_CONTINUE_ENABLED =
+      "http-client.apache.expect-continue-enabled";
+  /**
+   * Used to configure the max connections number for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections";
+  /**
+   * Used to configure whether to enable the tcp keep alive setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_TCP_KEEP_ALIVE_ENABLED =
+      "http-client.apache.tcp-keep-alive-enabled";
+  /**
+   * Used to configure whether to use idle connection reaper for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is enabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED =
+      "http-client.apache.use-idle-connection-reaper-enabled";
+
+  private String httpClientType;
+  private final Map<String, String> httpClientProperties;
+
+  public HttpClientProperties() {
+    this.httpClientType = CLIENT_TYPE_DEFAULT;
+    this.httpClientProperties = Collections.emptyMap();
+  }
+
+  public HttpClientProperties(Map<String, String> properties) {
+    this.httpClientType =
+        PropertyUtil.propertyAsString(properties, CLIENT_TYPE, CLIENT_TYPE_DEFAULT);
+    this.httpClientProperties =
+        PropertyUtil.filterProperties(properties, key -> key.startsWith(CLIENT_PREFIX));
+  }
+
+  /**
+   * Configure the httpClient for a client according to the HttpClientType. The two supported
+   * HttpClientTypes are urlconnection and apache
+   *
+   * <p>Sample usage:

Review Comment:
   I believe we don't need to close the paragraph tag`<\p>`? See - [oracle.com/technical-resources/articles/java/javadoc-tool.html#format ](https://www.oracle.com/technical-resources/articles/java/javadoc-tool.html#format)
   and https://stackoverflow.com/a/47690366 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189042242


##########
aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
+
+public class HttpClientProperties {
+
+  /**
+   * The type of {@link software.amazon.awssdk.http.SdkHttpClient} implementation used by {@link
+   * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
+   * {@link #CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see CLIENT_TYPE_*
+   * defined below.
+   */
+  public static final String CLIENT_TYPE = "http-client.type";
+
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
+   * AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_APACHE = "apache";
+
+  private static final String CLIENT_PREFIX = "http-client.";
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
+   * Client in {@link AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection";
+
+  public static final String CLIENT_TYPE_DEFAULT = CLIENT_TYPE_APACHE;
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS =
+      "http-client.urlconnection.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_SOCKET_TIMEOUT_MS =
+      "http-client.urlconnection.socket-timeout-ms";
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIMEOUT_MS =
+      "http-client.apache.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms";
+  /**
+   * Used to configure the connection acquisition timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS =
+      "http-client.apache.connection-acquisition-timeout-ms";
+  /**
+   * Used to configure the connection max idle time in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS =
+      "http-client.apache.connection-max-idle-time-ms";
+  /**
+   * Used to configure the connection time to live in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS =
+      "http-client.apache.connection-time-to-live-ms";
+  /**
+   * Used to configure whether to enable the expect continue setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_EXPECT_CONTINUE_ENABLED =
+      "http-client.apache.expect-continue-enabled";
+  /**
+   * Used to configure the max connections number for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections";
+  /**
+   * Used to configure whether to enable the tcp keep alive setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_TCP_KEEP_ALIVE_ENABLED =
+      "http-client.apache.tcp-keep-alive-enabled";
+  /**
+   * Used to configure whether to use idle connection reaper for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is enabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED =
+      "http-client.apache.use-idle-connection-reaper-enabled";
+
+  private String httpClientType;
+  private final Map<String, String> httpClientProperties;
+
+  public HttpClientProperties() {
+    this.httpClientType = CLIENT_TYPE_DEFAULT;
+    this.httpClientProperties = Collections.emptyMap();
+  }
+
+  public HttpClientProperties(Map<String, String> properties) {
+    this.httpClientType =
+        PropertyUtil.propertyAsString(properties, CLIENT_TYPE, CLIENT_TYPE_DEFAULT);
+    this.httpClientProperties =
+        PropertyUtil.filterProperties(properties, key -> key.startsWith(CLIENT_PREFIX));
+  }
+
+  /**
+   * Configure the httpClient for a client according to the HttpClientType. The two supported
+   * HttpClientTypes are urlconnection and apache
+   *
+   * <p>Sample usage:

Review Comment:
   Closing tag for the paragraph ? this issue exists in a few other places in this change, double check if you don't mind! at least for this case, I don't even think we need it here



##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -493,24 +493,37 @@ public class AwsProperties implements Serializable {
    * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
    * {@link #HTTP_CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see
    * HTTP_CLIENT_TYPE_* defined below.
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE = "http-client.type";
+  @Deprecated public static final String HTTP_CLIENT_TYPE = "http-client.type";
 
   /**
    * If this is set under {@link #HTTP_CLIENT_TYPE}, {@link
    * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
    * Client in {@link AwsClientFactory}
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE_URLCONNECTION = "urlconnection";
+  @Deprecated public static final String HTTP_CLIENT_TYPE_URLCONNECTION = "urlconnection";
 
   /**
    * If this is set under {@link #HTTP_CLIENT_TYPE}, {@link
    * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
    * AwsClientFactory}
+   *
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead
    */
-  public static final String HTTP_CLIENT_TYPE_APACHE = "apache";
+  @Deprecated public static final String HTTP_CLIENT_TYPE_APACHE = "apache";
 
-  public static final String HTTP_CLIENT_TYPE_DEFAULT = HTTP_CLIENT_TYPE_APACHE;
+  /**
+   * @deprecated will be removed in 1.4.0, use {@link org.apache.iceberg.aws.HttpClientProperties}
+   *     instead

Review Comment:
   The indentation for `instead` on the newline (and in all the other places) looks off to me, is this how spotless is formatting it? 



##########
aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
+
+public class HttpClientProperties {
+
+  /**
+   * The type of {@link software.amazon.awssdk.http.SdkHttpClient} implementation used by {@link
+   * AwsClientFactory} If set, all AWS clients will use this specified HTTP client. If not set,
+   * {@link #CLIENT_TYPE_DEFAULT} will be used. For specific types supported, see CLIENT_TYPE_*
+   * defined below.
+   */
+  public static final String CLIENT_TYPE = "http-client.type";
+
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client in {@link
+   * AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_APACHE = "apache";
+
+  private static final String CLIENT_PREFIX = "http-client.";
+  /**
+   * If this is set under {@link #CLIENT_TYPE}, {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP
+   * Client in {@link AwsClientFactory}
+   */
+  public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection";
+
+  public static final String CLIENT_TYPE_DEFAULT = CLIENT_TYPE_APACHE;
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS =
+      "http-client.urlconnection.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only
+   * works when {@link #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_URLCONNECTION}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/urlconnection/UrlConnectionHttpClient.Builder.html
+   */
+  public static final String URLCONNECTION_SOCKET_TIMEOUT_MS =
+      "http-client.urlconnection.socket-timeout-ms";
+  /**
+   * Used to configure the connection timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIMEOUT_MS =
+      "http-client.apache.connection-timeout-ms";
+  /**
+   * Used to configure the socket timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms";
+  /**
+   * Used to configure the connection acquisition timeout in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS =
+      "http-client.apache.connection-acquisition-timeout-ms";
+  /**
+   * Used to configure the connection max idle time in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS =
+      "http-client.apache.connection-max-idle-time-ms";
+  /**
+   * Used to configure the connection time to live in milliseconds for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS =
+      "http-client.apache.connection-time-to-live-ms";
+  /**
+   * Used to configure whether to enable the expect continue setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_EXPECT_CONTINUE_ENABLED =
+      "http-client.apache.expect-continue-enabled";
+  /**
+   * Used to configure the max connections number for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections";
+  /**
+   * Used to configure whether to enable the tcp keep alive setting for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is disabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_TCP_KEEP_ALIVE_ENABLED =
+      "http-client.apache.tcp-keep-alive-enabled";
+  /**
+   * Used to configure whether to use idle connection reaper for {@link
+   * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link
+   * #CLIENT_TYPE} is set to {@link #CLIENT_TYPE_APACHE}.
+   *
+   * <p>In default, this is enabled.
+   *
+   * <p>For more details, see
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html
+   */
+  public static final String APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED =
+      "http-client.apache.use-idle-connection-reaper-enabled";
+
+  private String httpClientType;
+  private final Map<String, String> httpClientProperties;
+
+  public HttpClientProperties() {
+    this.httpClientType = CLIENT_TYPE_DEFAULT;
+    this.httpClientProperties = Collections.emptyMap();
+  }
+
+  public HttpClientProperties(Map<String, String> properties) {
+    this.httpClientType =
+        PropertyUtil.propertyAsString(properties, CLIENT_TYPE, CLIENT_TYPE_DEFAULT);
+    this.httpClientProperties =
+        PropertyUtil.filterProperties(properties, key -> key.startsWith(CLIENT_PREFIX));
+  }
+
+  /**
+   * Configure the httpClient for a client according to the HttpClientType. The two supported
+   * HttpClientTypes are urlconnection and apache
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyHttpClientConfigurations)
+   * </pre>
+   */
+  public <T extends AwsSyncClientBuilder> void applyHttpClientConfigurations(T builder) {
+    if (Strings.isNullOrEmpty(httpClientType)) {
+      httpClientType = CLIENT_TYPE_DEFAULT;
+    }

Review Comment:
   Nit: Newline after `if` blocks



##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {
+      if (sessionToken == null) {
+        return StaticCredentialsProvider.create(
+            AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+      } else {
+        return StaticCredentialsProvider.create(
+            AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+      }
+    }
+
+    if (!Strings.isNullOrEmpty(this.clientCredentialsProvider)) {
+      return credentialsProvider(this.clientCredentialsProvider);
+    }
+
+    return DefaultCredentialsProvider.create();
+  }
+
+  private AwsCredentialsProvider credentialsProvider(String credentialsProviderClass) {
+    Class<?> providerClass;
+    try {
+      providerClass = DynClasses.builder().impl(credentialsProviderClass).buildChecked();
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Cannot load class %s, it does not exist in the classpath", credentialsProviderClass),
+          e);
+    }
+
+    Preconditions.checkArgument(
+        AwsCredentialsProvider.class.isAssignableFrom(providerClass),
+        String.format(
+            "Cannot initialize %s, it does not implement %s.",
+            credentialsProviderClass, AwsCredentialsProvider.class.getName()));
+
+    AwsCredentialsProvider provider;
+    try {
+      try {
+        provider =
+            DynMethods.builder("create")
+                .hiddenImpl(providerClass, Map.class)
+                .buildStaticChecked()
+                .invoke(clientCredentialsProviderProperties);
+      } catch (NoSuchMethodException e) {
+        provider =
+            DynMethods.builder("create").hiddenImpl(providerClass).buildStaticChecked().invoke();
+      }
+
+      return provider;
+    } catch (NoSuchMethodException e) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Cannot create an instance of %s, it does not contain a static 'create' or 'create(Map<String, String>)' method",
+              credentialsProviderClass),
+          e);
+    }

Review Comment:
   This block is a bit hard to read imo, I got what's going on but while we're at this refactoring we can take the opportunity to do some cleanup. For this could we have a separate helper for obtaining the provider with some inline comment on our "fallback" behavior.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189031910


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";

Review Comment:
   I think `client.region` also should be moved here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar merged pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar merged PR #7562:
URL: https://github.com/apache/iceberg/pull/7562


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7562: AWS: create HttpClientProperties, move s3 related methods into S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on code in PR #7562:
URL: https://github.com/apache/iceberg/pull/7562#discussion_r1189100789


##########
aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iceberg.aws;
+
+import java.util.Map;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.regions.Region;
+
+public class AwsClientProperties {
+  /**
+   * Configure the AWS credentials provider used to create AWS clients. A fully qualified concrete
+   * class with package that implements the {@link AwsCredentialsProvider} interface is required.
+   *
+   * <p>Additionally, the implementation class must also have a create() or create(Map) method
+   * implemented, which returns an instance of the class that provides aws credentials provider.
+   *
+   * <p>Example:
+   * client.credentials-provider=software.amazon.awssdk.auth.credentials.SystemPropertyCredentialsProvider
+   *
+   * <p>When set, the default client factory {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes will use this provider to get AWS credentials provided instead of reading the default
+   * credential chain to get AWS access credentials.
+   */
+  public static final String CLIENT_CREDENTIALS_PROVIDER = "client.credentials-provider";
+
+  /**
+   * Used by the client.credentials-provider configured value that will be used by {@link
+   * org.apache.iceberg.aws.AwsClientFactories#defaultFactory()} and other AWS client factory
+   * classes to pass provider-specific properties. Each property consists of a key name and an
+   * associated value.
+   */
+  private static final String CLIENT_CREDENTIAL_PROVIDER_PREFIX = "client.credentials-provider.";
+
+  /**
+   * Used by {@link org.apache.iceberg.aws.AwsClientFactories.DefaultAwsClientFactory} and also
+   * other client factory classes. If set, all AWS clients except STS client will use the given
+   * region instead of the default region chain.
+   */
+  public static final String CLIENT_REGION = "client.region";
+
+  private String clientRegion;
+  private String clientCredentialsProvider;
+  private final Map<String, String> clientCredentialsProviderProperties;
+
+  public AwsClientProperties() {
+    this.clientRegion = null;
+    this.clientCredentialsProvider = null;
+    this.clientCredentialsProviderProperties = null;
+  }
+
+  public AwsClientProperties(Map<String, String> properties) {
+    this.clientRegion = properties.get(CLIENT_REGION);
+    this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER);
+    this.clientCredentialsProviderProperties =
+        PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX);
+  }
+
+  public String clientRegion() {
+    return clientRegion;
+  }
+
+  public void setClientRegion(String clientRegion) {
+    this.clientRegion = clientRegion;
+  }
+
+  /**
+   * Configure a client AWS region.
+   *
+   * <p>Sample usage:
+   *
+   * <pre>
+   *     S3Client.builder().applyMutation(awsProperties::applyClientRegionConfiguration)
+   * </pre>
+   */
+  public <T extends AwsClientBuilder> void applyClientRegionConfiguration(T builder) {
+    if (clientRegion != null) {
+      builder.region(Region.of(clientRegion));
+    }
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  public AwsCredentialsProvider credentialsProvider(
+      String accessKeyId, String secretAccessKey, String sessionToken) {
+    if (accessKeyId != null) {

Review Comment:
   and `secretAccessKey` not null



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org