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/03 02:28:39 UTC

[GitHub] [iceberg] akshayakp97 opened a new pull request, #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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

   This PR moves all `S3FileIO` related properties into separate class, `S3FileIOProperties`.
   Breaking down this refactor work into multiple PRs.
   First, create `S3FileIOProperties` class which will be the source of truth for all `S3FileIO` related properties.
   `TestS3FileIOProperties.java` includes new test cases + lift-and-shift from `TestAwsProperties.class`
   All the code that makes references to these fields in `AwsProperties.java` will be updated in a future PR.


-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -69,46 +69,46 @@ public class AwsProperties implements Serializable {
    *
    * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
    */
-  public static final String S3FILEIO_SSE_TYPE = "s3.sse.type";
+  public static final String S3FILEIO_SSE_TYPE = S3FileIOProperties.S3FILEIO_SSE_TYPE;

Review Comment:
   I think we can just mark these properties as deprecated? Not sure if we should remove it at 2.0 or 1.4 though, given this is a pretty big change, 2.0 is probably better?



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.s3;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.aws.glue.GlueCatalog;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class S3FileIOProperties implements Serializable {
+
+  /**
+   * Type of S3 Server side encryption used, default to {@link
+   * S3FileIOProperties#S3FILEIO_SSE_TYPE_NONE}.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE = "s3.sse.type";
+
+  /** No server side encryption. */
+  public static final String S3FILEIO_SSE_TYPE_NONE = "none";
+
+  /**
+   * S3 SSE-KMS encryption.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_KMS = "kms";
+
+  /**
+   * S3 SSE-S3 encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_S3 = "s3";
+
+  /**
+   * S3 SSE-C encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_CUSTOM = "custom";
+
+  /**
+   * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not
+   * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64
+   * AES256 symmetric key.
+   */
+  public static final String S3FILEIO_SSE_KEY = "s3.sse.key";
+
+  /**
+   * If S3 encryption type is SSE-C, input is the base-64 MD5 digest of the secret key. This MD5
+   * must be explicitly passed in by the caller to ensure key integrity.
+   */
+  public static final String S3FILEIO_SSE_MD5 = "s3.sse.md5";
+
+  /**
+   * Number of threads to use for uploading parts to S3 (shared pool across all output streams),
+   * default to {@link Runtime#availableProcessors()}
+   */
+  public static final String S3FILEIO_MULTIPART_UPLOAD_THREADS = "s3.multipart.num-threads";
+
+  /**
+   * The size of a single part for multipart upload requests in bytes (default: 32MB). based on S3
+   * requirement, the part size must be at least 5MB. Too ensure performance of the reader and
+   * writer, the part size must be less than 2GB.
+   *
+   * <p>For more details, see https://docs.aws.amazon.com/AmazonS3/latest/dev/qfacts.html
+   */
+  public static final String S3FILEIO_MULTIPART_SIZE = "s3.multipart.part-size-bytes";
+
+  public static final int S3FILEIO_MULTIPART_SIZE_DEFAULT = 32 * 1024 * 1024;
+  public static final int S3FILEIO_MULTIPART_SIZE_MIN = 5 * 1024 * 1024;
+
+  /**
+   * The threshold expressed as a factor times the multipart size at which to switch from uploading
+   * using a single put object request to uploading using multipart upload (default: 1.5).
+   */
+  public static final String S3FILEIO_MULTIPART_THRESHOLD_FACTOR = "s3.multipart.threshold";
+
+  public static final double S3FILEIO_MULTIPART_THRESHOLD_FACTOR_DEFAULT = 1.5;
+
+  /**
+   * Location to put staging files for upload to S3, default to temp directory set in
+   * java.io.tmpdir.
+   */
+  public static final String S3FILEIO_STAGING_DIRECTORY = "s3.staging-dir";
+
+  /**
+   * Used to configure canned access control list (ACL) for S3 client to use during write. If not
+   * set, ACL will not be set for requests.
+   *
+   * <p>The input must be one of {@link software.amazon.awssdk.services.s3.model.ObjectCannedACL},
+   * such as 'public-read-write' For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html
+   */
+  public static final String S3FILEIO_ACL = "s3.acl";
+
+  /**
+   * Configure an alternative endpoint of the S3 service for S3FileIO to access.
+   *
+   * <p>This could be used to use S3FileIO with any s3-compatible object storage service that has a
+   * different endpoint, or access a private S3 endpoint in a virtual private cloud.
+   */
+  public static final String S3FILEIO_ENDPOINT = "s3.endpoint";
+
+  /**
+   * If set {@code true}, requests to S3FileIO will use Path-Style, otherwise, Virtual Hosted-Style
+   * will be used.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html
+   */
+  public static final String S3FILEIO_PATH_STYLE_ACCESS = "s3.path-style-access";
+
+  public static final boolean S3FILEIO_PATH_STYLE_ACCESS_DEFAULT = false;
+
+  /**
+   * Configure the static access key ID used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #S3FILEIO_SESSION_TOKEN} is set, session credential is used, otherwise basic credential is
+   * used.
+   */
+  public static final String S3FILEIO_ACCESS_KEY_ID = "s3.access-key-id";
+
+  /**
+   * Configure the static secret access key used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #S3FILEIO_SESSION_TOKEN} is set, session credential is used, otherwise basic credential is
+   * used.
+   */
+  public static final String S3FILEIO_SECRET_ACCESS_KEY = "s3.secret-access-key";
+
+  /**
+   * Configure the static session token used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the session credentials provided instead of
+   * reading the default credential chain to create S3 access credentials.
+   */
+  public static final String S3FILEIO_SESSION_TOKEN = "s3.session-token";
+
+  /**
+   * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an
+   * access point.
+   *
+   * <p>By default, attempting to use an access point in a different region will throw an exception.
+   * When enabled, this property allows using access points in other regions.
+   *
+   * <p>For more details see:
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/services/s3/S3Configuration.html#useArnRegionEnabled--
+   */
+  public static final String S3_USE_ARN_REGION_ENABLED = "s3.use-arn-region-enabled";
+
+  public static final boolean S3_USE_ARN_REGION_ENABLED_DEFAULT = false;
+
+  /** Enables eTag checks for S3 PUT and MULTIPART upload requests. */
+  public static final String S3_CHECKSUM_ENABLED = "s3.checksum-enabled";
+
+  public static final boolean S3_CHECKSUM_ENABLED_DEFAULT = false;
+
+  public static final String S3_REMOTE_SIGNING_ENABLED = "s3.remote-signing-enabled";
+
+  public static final boolean S3_REMOTE_SIGNING_ENABLED_DEFAULT = false;
+
+  /** Configure the batch size used when deleting multiple files from a given S3 bucket */
+  public static final String S3FILEIO_DELETE_BATCH_SIZE = "s3.delete.batch-size";
+
+  /**
+   * Default batch size used when deleting files.
+   *
+   * <p>Refer to https://github.com/apache/hadoop/commit/56dee667707926f3796c7757be1a133a362f05c9
+   * for more details on why this value was chosen.
+   */
+  public static final int S3FILEIO_DELETE_BATCH_SIZE_DEFAULT = 250;
+
+  /**
+   * Max possible batch size for deletion. Currently, a max of 1000 keys can be deleted in one
+   * batch. https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
+   */
+  public static final int S3FILEIO_DELETE_BATCH_SIZE_MAX = 1000;
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when writing. To set, we can pass a catalog property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.tags.my_key=my_val
+   */
+  public static final String S3_WRITE_TAGS_PREFIX = "s3.write.tags.";
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.table-tag-enabled=true
+   */
+  public static final String S3_WRITE_TABLE_TAG_ENABLED = "s3.write.table-tag-enabled";
+
+  public static final boolean S3_WRITE_TABLE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.namespace-tag-enabled=true
+   */
+  public static final String S3_WRITE_NAMESPACE_TAG_ENABLED = "s3.write.namespace-tag-enabled";
+
+  public static final boolean S3_WRITE_NAMESPACE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Tag name that will be used by {@link #S3_WRITE_TAGS_PREFIX} when {@link
+   * #S3_WRITE_TABLE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.table=tableName
+   */
+  public static final String S3_TAG_ICEBERG_TABLE = "iceberg.table";
+
+  /**
+   * Tag name that will be used by {@link #S3_WRITE_TAGS_PREFIX} when {@link
+   * #S3_WRITE_NAMESPACE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.namespace=namespaceName
+   */
+  public static final String S3_TAG_ICEBERG_NAMESPACE = "iceberg.namespace";
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when deleting. When this config is set, objects are
+   * tagged with the configured key-value pairs before deletion. This is considered a soft-delete,
+   * because users are able to configure tag-based object lifecycle policy at bucket level to
+   * transition objects to different tiers.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-lifecycle-mgmt.html
+   *
+   * <p>Example: s3.delete.tags.my_key=my_val
+   */
+  public static final String S3_DELETE_TAGS_PREFIX = "s3.delete.tags.";
+
+  /**
+   * Number of threads to use for adding delete tags to S3 objects, default to {@link
+   * Runtime#availableProcessors()}
+   */
+  public static final String S3FILEIO_DELETE_THREADS = "s3.delete.num-threads";
+
+  /**
+   * Determines if {@link S3FileIO} deletes the object when io.delete() is called, default to true.
+   * Once disabled, users are expected to set tags through {@link #S3_DELETE_TAGS_PREFIX} and manage
+   * deleted files through S3 lifecycle policy.
+   */
+  public static final String S3_DELETE_ENABLED = "s3.delete-enabled";
+
+  public static final boolean S3_DELETE_ENABLED_DEFAULT = true;
+
+  /**
+   * Determines if S3 client will use the Acceleration Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/transfer-acceleration.html
+   */
+  public static final String S3_ACCELERATION_ENABLED = "s3.acceleration-enabled";
+
+  public static final boolean S3_ACCELERATION_ENABLED_DEFAULT = false;
+
+  /**
+   * Determines if S3 client will use the Dualstack Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/dual-stack-endpoints.html
+   */
+  public static final String S3_DUALSTACK_ENABLED = "s3.dualstack-enabled";
+
+  public static final boolean S3_DUALSTACK_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can
+   * pass a catalog property.
+   *
+   * <p>For more details, see https://aws.amazon.com/s3/features/access-points/
+   *
+   * <p>Example: s3.access-points.my-bucket=access-point
+   */
+  public static final String S3_ACCESS_POINTS_PREFIX = "s3.access-points.";
+
+  /**
+   * This flag controls whether the S3 client will be initialized during the S3FileIO
+   * initialization, instead of default lazy initialization upon use. This is needed for cases that
+   * the credentials to use might change and needs to be preloaded.
+   */
+  public static final String S3_PRELOAD_CLIENT_ENABLED = "s3.preload-client-enabled";
+
+  public static final boolean S3_PRELOAD_CLIENT_ENABLED_DEFAULT = false;
+
+  private String s3FileIoSseType;

Review Comment:
   can we take this chance to standardize the variable names? currently there is a mixture of `s3FileIoXXX` and `s3XXXX`, I think they should all match the config names and be `s3XXX`.



-- 
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] nastra commented on a diff in pull request #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.s3;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class TestS3FileIOProperties {
+  private static final String S3_TEST_BUCKET_NAME = "my_bucket";
+  private static final String S3_TEST_BUCKET_ACCESS_POINT = "access_point";
+  private static final String S3_WRITE_TAG_KEY = "my_key";
+  private static final String S3_WRITE_TAG_VALUE = "my_value";
+  private static final String S3_DELETE_TAG_KEY = "my_key";
+  private static final String S3_DELETE_TAG_VALUE = "my_value";
+
+  @Test
+  public void testS3FileIOPropertiesDefaultValues() {
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties();
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3FILEIO_SSE_TYPE_NONE, s3FileIOProperties.s3FileIoSseType());
+
+    Assert.assertNull(s3FileIOProperties.s3FileIoSseKey());
+    Assert.assertNull(s3FileIOProperties.s3FileIoSseMd5());
+    Assert.assertNull(s3FileIOProperties.s3AccessKeyId());
+    Assert.assertNull(s3FileIOProperties.s3SecretAccessKey());
+    Assert.assertNull(s3FileIOProperties.s3SessionToken());
+    Assert.assertNull(s3FileIOProperties.s3FileIoAcl());
+    Assert.assertNull(s3FileIOProperties.s3Endpoint());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_PRELOAD_CLIENT_ENABLED_DEFAULT,
+        s3FileIOProperties.s3PreloadClientEnabled());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_DUALSTACK_ENABLED_DEFAULT, s3FileIOProperties.isS3DualStackEnabled());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3FILEIO_PATH_STYLE_ACCESS_DEFAULT,
+        s3FileIOProperties.isS3PathStyleAccess());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_USE_ARN_REGION_ENABLED_DEFAULT,
+        s3FileIOProperties.isS3UseArnRegionEnabled());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_ACCELERATION_ENABLED_DEFAULT,
+        s3FileIOProperties.isS3AccelerationEnabled());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_REMOTE_SIGNING_ENABLED_DEFAULT,
+        s3FileIOProperties.isS3RemoteSigningEnabled());
+
+    Assert.assertEquals(
+        Runtime.getRuntime().availableProcessors(),
+        s3FileIOProperties.s3FileIoMultipartUploadThreads());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3FILEIO_MULTIPART_SIZE_DEFAULT,
+        s3FileIOProperties.s3FileIoMultiPartSize());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3FILEIO_MULTIPART_THRESHOLD_FACTOR_DEFAULT,
+        s3FileIOProperties.s3FileIOMultipartThresholdFactor(),
+        0.0);
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3FILEIO_DELETE_BATCH_SIZE_DEFAULT,
+        s3FileIOProperties.s3FileIoDeleteBatchSize(),
+        0.0);
+
+    Assert.assertEquals(
+        System.getProperty("java.io.tmpdir"), s3FileIOProperties.s3fileIoStagingDirectory());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_CHECKSUM_ENABLED_DEFAULT, s3FileIOProperties.isS3ChecksumEnabled());
+
+    Assert.assertEquals(Sets.newHashSet(), s3FileIOProperties.s3WriteTags());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_WRITE_TABLE_TAG_ENABLED_DEFAULT,
+        s3FileIOProperties.s3WriteTableTagEnabled());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_WRITE_NAMESPACE_TAG_ENABLED_DEFAULT,
+        s3FileIOProperties.s3WriteNamespaceTagEnabled());
+
+    Assert.assertEquals(Sets.newHashSet(), s3FileIOProperties.s3DeleteTags());
+
+    Assert.assertEquals(
+        Runtime.getRuntime().availableProcessors(), s3FileIOProperties.s3FileIoDeleteThreads());
+
+    Assert.assertEquals(
+        S3FileIOProperties.S3_DELETE_ENABLED_DEFAULT, s3FileIOProperties.isS3DeleteEnabled());
+
+    Assert.assertEquals(Collections.emptyMap(), s3FileIOProperties.s3BucketToAccessPointMapping());
+  }
+
+  @Test
+  public void testS3FileIOProperties() {
+    Map<String, String> map = getTestProperties();
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(map);
+
+    Assert.assertEquals(

Review Comment:
   I think a slightly better alternative to testing this would be: `Assertions.assertThat(map).containsEntry(S3FileIOProperties.S3FILEIO_SSE_TYPE), s3FileIOProperties.s3FileIoSseType())`.
   
   The main advantage here is that it will show the content of `map` in case the assertion ever fails.



##########
aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.s3;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;

Review Comment:
   given that we're adding a new test class I think it makes sense to use JUnit5 rather than JUnit4 tests here and potentially also switch to AssertJ rather than using `Assert` from JUnit4 as this will make a future port of the codebase to JUnit5 easier. See also https://iceberg.apache.org/contribute/#testing for some additional information on these topics



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -985,35 +933,35 @@ public String clientAssumeRoleSessionName() {
   }
 
   public String s3FileIoSseType() {
-    return s3FileIoSseType;
+    return s3FileIOProperties.s3FileIoSseType();

Review Comment:
   > Overall my thought is that we should not have nested properties
   
   So, we don’t want to make any changes to the methods is it? If so, then I have to add back the class fields to AwsProperties.
   Which one do we want?
   ```
   
   public String s3FileIoSseType() {
       return s3FileIoSseType;
   }
   ```
     OR
   
   ```
   public String s3FileIoSseType() {
     return s3FileIOProperties.s3FileIoSseType();
   }
   ```



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -68,47 +68,63 @@ public class AwsProperties implements Serializable {
    * AwsProperties#S3FILEIO_SSE_TYPE_NONE}.
    *
    * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   *
+   * @deprecated use {@link org.apache.iceberg.aws.s3.S3FileIOProperties} instead

Review Comment:
   nit: should also add "will be removed in Iceberg x.x". But I am not sure what's the best version to remove it tbh. @amogh-jahagirdar @nastra any thoughts?



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -985,35 +933,35 @@ public String clientAssumeRoleSessionName() {
   }
 
   public String s3FileIoSseType() {
-    return s3FileIoSseType;
+    return s3FileIOProperties.s3FileIoSseType();

Review Comment:
   similar to the comment about variables, we should also just mark these methods as deprecated, with javadoc saying which version will this be removed. Overall my thought is that we should not have nested properties. We will have fully separated properties classes, similar to the `XXXConfig` in Trino, that seems to be pretty neat.



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.s3;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.aws.glue.GlueCatalog;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class S3FileIOProperties implements Serializable {
+
+  /**
+   * Type of S3 Server side encryption used, default to {@link
+   * S3FileIOProperties#S3FILEIO_SSE_TYPE_NONE}.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE = "s3.sse.type";
+
+  /** No server side encryption. */
+  public static final String S3FILEIO_SSE_TYPE_NONE = "none";
+
+  /**
+   * S3 SSE-KMS encryption.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_KMS = "kms";
+
+  /**
+   * S3 SSE-S3 encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_S3 = "s3";
+
+  /**
+   * S3 SSE-C encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
+   */
+  public static final String S3FILEIO_SSE_TYPE_CUSTOM = "custom";
+
+  /**
+   * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not
+   * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64
+   * AES256 symmetric key.
+   */
+  public static final String S3FILEIO_SSE_KEY = "s3.sse.key";
+
+  /**
+   * If S3 encryption type is SSE-C, input is the base-64 MD5 digest of the secret key. This MD5
+   * must be explicitly passed in by the caller to ensure key integrity.
+   */
+  public static final String S3FILEIO_SSE_MD5 = "s3.sse.md5";
+
+  /**
+   * Number of threads to use for uploading parts to S3 (shared pool across all output streams),
+   * default to {@link Runtime#availableProcessors()}
+   */
+  public static final String S3FILEIO_MULTIPART_UPLOAD_THREADS = "s3.multipart.num-threads";
+
+  /**
+   * The size of a single part for multipart upload requests in bytes (default: 32MB). based on S3
+   * requirement, the part size must be at least 5MB. Too ensure performance of the reader and
+   * writer, the part size must be less than 2GB.
+   *
+   * <p>For more details, see https://docs.aws.amazon.com/AmazonS3/latest/dev/qfacts.html
+   */
+  public static final String S3FILEIO_MULTIPART_SIZE = "s3.multipart.part-size-bytes";
+
+  public static final int S3FILEIO_MULTIPART_SIZE_DEFAULT = 32 * 1024 * 1024;
+  public static final int S3FILEIO_MULTIPART_SIZE_MIN = 5 * 1024 * 1024;
+
+  /**
+   * The threshold expressed as a factor times the multipart size at which to switch from uploading
+   * using a single put object request to uploading using multipart upload (default: 1.5).
+   */
+  public static final String S3FILEIO_MULTIPART_THRESHOLD_FACTOR = "s3.multipart.threshold";
+
+  public static final double S3FILEIO_MULTIPART_THRESHOLD_FACTOR_DEFAULT = 1.5;
+
+  /**
+   * Location to put staging files for upload to S3, default to temp directory set in
+   * java.io.tmpdir.
+   */
+  public static final String S3FILEIO_STAGING_DIRECTORY = "s3.staging-dir";
+
+  /**
+   * Used to configure canned access control list (ACL) for S3 client to use during write. If not
+   * set, ACL will not be set for requests.
+   *
+   * <p>The input must be one of {@link software.amazon.awssdk.services.s3.model.ObjectCannedACL},
+   * such as 'public-read-write' For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html
+   */
+  public static final String S3FILEIO_ACL = "s3.acl";
+
+  /**
+   * Configure an alternative endpoint of the S3 service for S3FileIO to access.
+   *
+   * <p>This could be used to use S3FileIO with any s3-compatible object storage service that has a
+   * different endpoint, or access a private S3 endpoint in a virtual private cloud.
+   */
+  public static final String S3FILEIO_ENDPOINT = "s3.endpoint";
+
+  /**
+   * If set {@code true}, requests to S3FileIO will use Path-Style, otherwise, Virtual Hosted-Style
+   * will be used.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html
+   */
+  public static final String S3FILEIO_PATH_STYLE_ACCESS = "s3.path-style-access";
+
+  public static final boolean S3FILEIO_PATH_STYLE_ACCESS_DEFAULT = false;
+
+  /**
+   * Configure the static access key ID used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #S3FILEIO_SESSION_TOKEN} is set, session credential is used, otherwise basic credential is
+   * used.
+   */
+  public static final String S3FILEIO_ACCESS_KEY_ID = "s3.access-key-id";
+
+  /**
+   * Configure the static secret access key used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #S3FILEIO_SESSION_TOKEN} is set, session credential is used, otherwise basic credential is
+   * used.
+   */
+  public static final String S3FILEIO_SECRET_ACCESS_KEY = "s3.secret-access-key";
+
+  /**
+   * Configure the static session token used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the session credentials provided instead of
+   * reading the default credential chain to create S3 access credentials.
+   */
+  public static final String S3FILEIO_SESSION_TOKEN = "s3.session-token";
+
+  /**
+   * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an
+   * access point.
+   *
+   * <p>By default, attempting to use an access point in a different region will throw an exception.
+   * When enabled, this property allows using access points in other regions.
+   *
+   * <p>For more details see:
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/services/s3/S3Configuration.html#useArnRegionEnabled--
+   */
+  public static final String S3_USE_ARN_REGION_ENABLED = "s3.use-arn-region-enabled";
+
+  public static final boolean S3_USE_ARN_REGION_ENABLED_DEFAULT = false;
+
+  /** Enables eTag checks for S3 PUT and MULTIPART upload requests. */
+  public static final String S3_CHECKSUM_ENABLED = "s3.checksum-enabled";
+
+  public static final boolean S3_CHECKSUM_ENABLED_DEFAULT = false;
+
+  public static final String S3_REMOTE_SIGNING_ENABLED = "s3.remote-signing-enabled";
+
+  public static final boolean S3_REMOTE_SIGNING_ENABLED_DEFAULT = false;
+
+  /** Configure the batch size used when deleting multiple files from a given S3 bucket */
+  public static final String S3FILEIO_DELETE_BATCH_SIZE = "s3.delete.batch-size";
+
+  /**
+   * Default batch size used when deleting files.
+   *
+   * <p>Refer to https://github.com/apache/hadoop/commit/56dee667707926f3796c7757be1a133a362f05c9
+   * for more details on why this value was chosen.
+   */
+  public static final int S3FILEIO_DELETE_BATCH_SIZE_DEFAULT = 250;
+
+  /**
+   * Max possible batch size for deletion. Currently, a max of 1000 keys can be deleted in one
+   * batch. https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
+   */
+  public static final int S3FILEIO_DELETE_BATCH_SIZE_MAX = 1000;
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when writing. To set, we can pass a catalog property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.tags.my_key=my_val
+   */
+  public static final String S3_WRITE_TAGS_PREFIX = "s3.write.tags.";
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.table-tag-enabled=true
+   */
+  public static final String S3_WRITE_TABLE_TAG_ENABLED = "s3.write.table-tag-enabled";
+
+  public static final boolean S3_WRITE_TABLE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.namespace-tag-enabled=true
+   */
+  public static final String S3_WRITE_NAMESPACE_TAG_ENABLED = "s3.write.namespace-tag-enabled";
+
+  public static final boolean S3_WRITE_NAMESPACE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Tag name that will be used by {@link #S3_WRITE_TAGS_PREFIX} when {@link
+   * #S3_WRITE_TABLE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.table=tableName
+   */
+  public static final String S3_TAG_ICEBERG_TABLE = "iceberg.table";
+
+  /**
+   * Tag name that will be used by {@link #S3_WRITE_TAGS_PREFIX} when {@link
+   * #S3_WRITE_NAMESPACE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.namespace=namespaceName
+   */
+  public static final String S3_TAG_ICEBERG_NAMESPACE = "iceberg.namespace";
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when deleting. When this config is set, objects are
+   * tagged with the configured key-value pairs before deletion. This is considered a soft-delete,
+   * because users are able to configure tag-based object lifecycle policy at bucket level to
+   * transition objects to different tiers.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-lifecycle-mgmt.html
+   *
+   * <p>Example: s3.delete.tags.my_key=my_val
+   */
+  public static final String S3_DELETE_TAGS_PREFIX = "s3.delete.tags.";
+
+  /**
+   * Number of threads to use for adding delete tags to S3 objects, default to {@link
+   * Runtime#availableProcessors()}
+   */
+  public static final String S3FILEIO_DELETE_THREADS = "s3.delete.num-threads";
+
+  /**
+   * Determines if {@link S3FileIO} deletes the object when io.delete() is called, default to true.
+   * Once disabled, users are expected to set tags through {@link #S3_DELETE_TAGS_PREFIX} and manage
+   * deleted files through S3 lifecycle policy.
+   */
+  public static final String S3_DELETE_ENABLED = "s3.delete-enabled";
+
+  public static final boolean S3_DELETE_ENABLED_DEFAULT = true;
+
+  /**
+   * Determines if S3 client will use the Acceleration Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/transfer-acceleration.html
+   */
+  public static final String S3_ACCELERATION_ENABLED = "s3.acceleration-enabled";
+
+  public static final boolean S3_ACCELERATION_ENABLED_DEFAULT = false;
+
+  /**
+   * Determines if S3 client will use the Dualstack Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/dual-stack-endpoints.html
+   */
+  public static final String S3_DUALSTACK_ENABLED = "s3.dualstack-enabled";
+
+  public static final boolean S3_DUALSTACK_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can
+   * pass a catalog property.
+   *
+   * <p>For more details, see https://aws.amazon.com/s3/features/access-points/
+   *
+   * <p>Example: s3.access-points.my-bucket=access-point
+   */
+  public static final String S3_ACCESS_POINTS_PREFIX = "s3.access-points.";
+
+  /**
+   * This flag controls whether the S3 client will be initialized during the S3FileIO
+   * initialization, instead of default lazy initialization upon use. This is needed for cases that
+   * the credentials to use might change and needs to be preloaded.
+   */
+  public static final String S3_PRELOAD_CLIENT_ENABLED = "s3.preload-client-enabled";
+
+  public static final boolean S3_PRELOAD_CLIENT_ENABLED_DEFAULT = false;
+
+  private String s3FileIoSseType;

Review Comment:
   Sorry, what do you mean by "match the config names"? If by config you mean `S3FileIOProperties`, then shouldn't we update all these variable names to `s3FileIoXXX`? 



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -1179,9 +1127,12 @@ public void setClientRegion(String clientRegion) {
    */
   public <T extends S3ClientBuilder> void applyS3CredentialConfigurations(T builder) {

Review Comment:
   for these classes, my thought is that we can make them static methods taking multiple properties classes. This probably means we should have another PR to at least separate out the `HttpClientProperties` and `AwsClientProperties`. Before that, we can keep these methods as is for now.



-- 
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 #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -0,0 +1,653 @@
+/*
+ * 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.s3;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.aws.glue.GlueCatalog;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class S3FileIOProperties implements Serializable {
+
+  /**
+   * Type of S3 Server side encryption used, default to {@link S3FileIOProperties#SSE_TYPE_NONE}.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   */
+  public static final String SSE_TYPE = "s3.sse.type";
+
+  /** No server side encryption. */
+  public static final String SSE_TYPE_NONE = "none";
+
+  /**
+   * S3 SSE-KMS encryption.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+   */
+  public static final String SSE_TYPE_KMS = "kms";
+
+  /**
+   * S3 SSE-S3 encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+   */
+  public static final String SSE_TYPE_S3 = "s3";
+
+  /**
+   * S3 SSE-C encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
+   */
+  public static final String SSE_TYPE_CUSTOM = "custom";
+
+  /**
+   * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not
+   * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64
+   * AES256 symmetric key.
+   */
+  public static final String SSE_KEY = "s3.sse.key";
+
+  /**
+   * If S3 encryption type is SSE-C, input is the base-64 MD5 digest of the secret key. This MD5
+   * must be explicitly passed in by the caller to ensure key integrity.
+   */
+  public static final String SSE_MD5 = "s3.sse.md5";
+
+  /**
+   * Number of threads to use for uploading parts to S3 (shared pool across all output streams),
+   * default to {@link Runtime#availableProcessors()}
+   */
+  public static final String MULTIPART_UPLOAD_THREADS = "s3.multipart.num-threads";
+
+  /**
+   * The size of a single part for multipart upload requests in bytes (default: 32MB). based on S3
+   * requirement, the part size must be at least 5MB. Too ensure performance of the reader and
+   * writer, the part size must be less than 2GB.
+   *
+   * <p>For more details, see https://docs.aws.amazon.com/AmazonS3/latest/dev/qfacts.html
+   */
+  public static final String MULTIPART_SIZE = "s3.multipart.part-size-bytes";
+
+  public static final int MULTIPART_SIZE_DEFAULT = 32 * 1024 * 1024;
+  public static final int MULTIPART_SIZE_MIN = 5 * 1024 * 1024;
+
+  /**
+   * The threshold expressed as a factor times the multipart size at which to switch from uploading
+   * using a single put object request to uploading using multipart upload (default: 1.5).
+   */
+  public static final String MULTIPART_THRESHOLD_FACTOR = "s3.multipart.threshold";
+
+  public static final double MULTIPART_THRESHOLD_FACTOR_DEFAULT = 1.5;
+
+  /**
+   * Location to put staging files for upload to S3, default to temp directory set in
+   * java.io.tmpdir.
+   */
+  public static final String STAGING_DIRECTORY = "s3.staging-dir";
+
+  /**
+   * Used to configure canned access control list (ACL) for S3 client to use during write. If not
+   * set, ACL will not be set for requests.
+   *
+   * <p>The input must be one of {@link software.amazon.awssdk.services.s3.model.ObjectCannedACL},
+   * such as 'public-read-write' For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html
+   */
+  public static final String ACL = "s3.acl";
+
+  /**
+   * Configure an alternative endpoint of the S3 service for S3FileIO to access.
+   *
+   * <p>This could be used to use S3FileIO with any s3-compatible object storage service that has a
+   * different endpoint, or access a private S3 endpoint in a virtual private cloud.
+   */
+  public static final String ENDPOINT = "s3.endpoint";
+
+  /**
+   * If set {@code true}, requests to S3FileIO will use Path-Style, otherwise, Virtual Hosted-Style
+   * will be used.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html
+   */
+  public static final String PATH_STYLE_ACCESS = "s3.path-style-access";
+
+  public static final boolean PATH_STYLE_ACCESS_DEFAULT = false;
+
+  /**
+   * Configure the static access key ID used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #SESSION_TOKEN} is set, session credential is used, otherwise basic credential is used.
+   */
+  public static final String ACCESS_KEY_ID = "s3.access-key-id";
+
+  /**
+   * Configure the static secret access key used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #SESSION_TOKEN} is set, session credential is used, otherwise basic credential is used.
+   */
+  public static final String SECRET_ACCESS_KEY = "s3.secret-access-key";
+
+  /**
+   * Configure the static session token used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the session credentials provided instead of
+   * reading the default credential chain to create S3 access credentials.
+   */
+  public static final String SESSION_TOKEN = "s3.session-token";
+
+  /**
+   * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an
+   * access point.
+   *
+   * <p>By default, attempting to use an access point in a different region will throw an exception.
+   * When enabled, this property allows using access points in other regions.
+   *
+   * <p>For more details see:
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/services/s3/S3Configuration.html#useArnRegionEnabled--
+   */
+  public static final String USE_ARN_REGION_ENABLED = "s3.use-arn-region-enabled";
+
+  public static final boolean USE_ARN_REGION_ENABLED_DEFAULT = false;
+
+  /** Enables eTag checks for S3 PUT and MULTIPART upload requests. */
+  public static final String CHECKSUM_ENABLED = "s3.checksum-enabled";
+
+  public static final boolean CHECKSUM_ENABLED_DEFAULT = false;
+
+  public static final String REMOTE_SIGNING_ENABLED = "s3.remote-signing-enabled";
+
+  public static final boolean REMOTE_SIGNING_ENABLED_DEFAULT = false;
+
+  /** Configure the batch size used when deleting multiple files from a given S3 bucket */
+  public static final String DELETE_BATCH_SIZE = "s3.delete.batch-size";
+
+  /**
+   * Default batch size used when deleting files.
+   *
+   * <p>Refer to https://github.com/apache/hadoop/commit/56dee667707926f3796c7757be1a133a362f05c9
+   * for more details on why this value was chosen.
+   */
+  public static final int DELETE_BATCH_SIZE_DEFAULT = 250;
+
+  /**
+   * Max possible batch size for deletion. Currently, a max of 1000 keys can be deleted in one
+   * batch. https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
+   */
+  public static final int DELETE_BATCH_SIZE_MAX = 1000;
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when writing. To set, we can pass a catalog property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.tags.my_key=my_val
+   */
+  public static final String WRITE_TAGS_PREFIX = "s3.write.tags.";
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.table-tag-enabled=true
+   */
+  public static final String WRITE_TABLE_TAG_ENABLED = "s3.write.table-tag-enabled";
+
+  public static final boolean WRITE_TABLE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.namespace-tag-enabled=true
+   */
+  public static final String WRITE_NAMESPACE_TAG_ENABLED = "s3.write.namespace-tag-enabled";
+
+  public static final boolean WRITE_NAMESPACE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Tag name that will be used by {@link #WRITE_TAGS_PREFIX} when {@link #WRITE_TABLE_TAG_ENABLED}
+   * is enabled
+   *
+   * <p>Example: iceberg.table=tableName
+   */
+  public static final String S3_TAG_ICEBERG_TABLE = "iceberg.table";
+
+  /**
+   * Tag name that will be used by {@link #WRITE_TAGS_PREFIX} when {@link
+   * #WRITE_NAMESPACE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.namespace=namespaceName
+   */
+  public static final String S3_TAG_ICEBERG_NAMESPACE = "iceberg.namespace";
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when deleting. When this config is set, objects are
+   * tagged with the configured key-value pairs before deletion. This is considered a soft-delete,
+   * because users are able to configure tag-based object lifecycle policy at bucket level to
+   * transition objects to different tiers.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-lifecycle-mgmt.html
+   *
+   * <p>Example: s3.delete.tags.my_key=my_val
+   */
+  public static final String DELETE_TAGS_PREFIX = "s3.delete.tags.";
+
+  /**
+   * Number of threads to use for adding delete tags to S3 objects, default to {@link
+   * Runtime#availableProcessors()}
+   */
+  public static final String DELETE_THREADS = "s3.delete.num-threads";
+
+  /**
+   * Determines if {@link S3FileIO} deletes the object when io.delete() is called, default to true.
+   * Once disabled, users are expected to set tags through {@link #DELETE_TAGS_PREFIX} and manage
+   * deleted files through S3 lifecycle policy.
+   */
+  public static final String DELETE_ENABLED = "s3.delete-enabled";
+
+  public static final boolean DELETE_ENABLED_DEFAULT = true;
+
+  /**
+   * Determines if S3 client will use the Acceleration Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/transfer-acceleration.html
+   */
+  public static final String ACCELERATION_ENABLED = "s3.acceleration-enabled";
+
+  public static final boolean ACCELERATION_ENABLED_DEFAULT = false;
+
+  /**
+   * Determines if S3 client will use the Dualstack Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/dual-stack-endpoints.html
+   */
+  public static final String DUALSTACK_ENABLED = "s3.dualstack-enabled";
+
+  public static final boolean DUALSTACK_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can
+   * pass a catalog property.
+   *
+   * <p>For more details, see https://aws.amazon.com/s3/features/access-points/
+   *
+   * <p>Example: s3.access-points.my-bucket=access-point
+   */
+  public static final String ACCESS_POINTS_PREFIX = "s3.access-points.";
+
+  /**
+   * This flag controls whether the S3 client will be initialized during the S3FileIO
+   * initialization, instead of default lazy initialization upon use. This is needed for cases that
+   * the credentials to use might change and needs to be preloaded.
+   */
+  public static final String PRELOAD_CLIENT_ENABLED = "s3.preload-client-enabled";
+
+  public static final boolean PRELOAD_CLIENT_ENABLED_DEFAULT = false;
+
+  private String sseType;
+  private String sseKey;
+  private String sseMd5;
+  private String accessKeyId;
+  private String secretAccessKey;
+  private String sessionToken;
+  private int multipartUploadThreads;
+  private int multiPartSize;
+  private int deleteBatchSize;
+  private double multipartThresholdFactor;
+  private String stagingDirectory;
+  private ObjectCannedACL acl;
+  private boolean isChecksumEnabled;
+  private final Set<Tag> writeTags;
+  private boolean isWriteTableTagEnabled;
+  private boolean isWriteNamespaceTagEnabled;
+  private final Set<Tag> deleteTags;
+  private int deleteThreads;
+  private boolean isDeleteEnabled;
+  private final Map<String, String> bucketToAccessPointMapping;
+  private boolean isPreloadClientEnabled;
+  private boolean isDualStackEnabled;
+  private boolean isPathStyleAccess;
+  private boolean isUseArnRegionEnabled;
+  private boolean isAccelerationEnabled;
+  private String endpoint;
+  private final boolean isRemoteSigningEnabled;
+
+  public S3FileIOProperties() {
+    this.sseType = SSE_TYPE_NONE;
+    this.sseKey = null;
+    this.sseMd5 = null;
+    this.accessKeyId = null;
+    this.secretAccessKey = null;
+    this.sessionToken = null;
+    this.acl = null;
+    this.endpoint = null;
+    this.multipartUploadThreads = Runtime.getRuntime().availableProcessors();
+    this.multiPartSize = MULTIPART_SIZE_DEFAULT;
+    this.multipartThresholdFactor = MULTIPART_THRESHOLD_FACTOR_DEFAULT;
+    this.deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT;
+    this.stagingDirectory = System.getProperty("java.io.tmpdir");
+    this.isChecksumEnabled = CHECKSUM_ENABLED_DEFAULT;
+    this.writeTags = Sets.newHashSet();
+    this.isWriteTableTagEnabled = WRITE_TABLE_TAG_ENABLED_DEFAULT;
+    this.isWriteNamespaceTagEnabled = WRITE_NAMESPACE_TAG_ENABLED_DEFAULT;
+    this.deleteTags = Sets.newHashSet();
+    this.deleteThreads = Runtime.getRuntime().availableProcessors();
+    this.isDeleteEnabled = DELETE_ENABLED_DEFAULT;
+    this.bucketToAccessPointMapping = Collections.emptyMap();
+    this.isPreloadClientEnabled = PRELOAD_CLIENT_ENABLED_DEFAULT;
+    this.isDualStackEnabled = DUALSTACK_ENABLED_DEFAULT;
+    this.isPathStyleAccess = PATH_STYLE_ACCESS_DEFAULT;
+    this.isUseArnRegionEnabled = USE_ARN_REGION_ENABLED_DEFAULT;
+    this.isAccelerationEnabled = ACCELERATION_ENABLED_DEFAULT;
+    this.isRemoteSigningEnabled = REMOTE_SIGNING_ENABLED_DEFAULT;
+
+    ValidationException.check(
+        keyIdAccessKeyBothConfigured(),
+        "S3 client access key ID and secret access key must be set at the same time");
+  }
+
+  public S3FileIOProperties(Map<String, String> properties) {
+    this.sseType = properties.getOrDefault(SSE_TYPE, SSE_TYPE_NONE);
+    this.sseKey = properties.get(SSE_KEY);
+    this.sseMd5 = properties.get(SSE_MD5);
+    this.accessKeyId = properties.get(ACCESS_KEY_ID);
+    this.secretAccessKey = properties.get(SECRET_ACCESS_KEY);
+    this.sessionToken = properties.get(SESSION_TOKEN);
+    if (SSE_TYPE_CUSTOM.equals(sseType)) {
+      Preconditions.checkNotNull(

Review Comment:
   +1



-- 
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 pull request #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

Posted by "jackye1995 (via GitHub)" <gi...@apache.org>.
jackye1995 commented on PR #7505:
URL: https://github.com/apache/iceberg/pull/7505#issuecomment-1535419372

   Thanks for the work, let me know when you have the follow up PR!


-- 
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 merged pull request #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


-- 
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] nastra commented on a diff in pull request #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -68,47 +68,70 @@ public class AwsProperties implements Serializable {
    * AwsProperties#S3FILEIO_SSE_TYPE_NONE}.
    *
    * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   *
+   * @deprecated will be removed in Iceberg 1.4, use {@link

Review Comment:
   nit: I think in other places we typically just mention the Iceberg version without `Iceberg`: `@deprecated will be removed in 1.4.0`



##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -0,0 +1,653 @@
+/*
+ * 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.s3;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.aws.glue.GlueCatalog;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class S3FileIOProperties implements Serializable {
+
+  /**
+   * Type of S3 Server side encryption used, default to {@link S3FileIOProperties#SSE_TYPE_NONE}.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   */
+  public static final String SSE_TYPE = "s3.sse.type";
+
+  /** No server side encryption. */
+  public static final String SSE_TYPE_NONE = "none";
+
+  /**
+   * S3 SSE-KMS encryption.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+   */
+  public static final String SSE_TYPE_KMS = "kms";
+
+  /**
+   * S3 SSE-S3 encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+   */
+  public static final String SSE_TYPE_S3 = "s3";
+
+  /**
+   * S3 SSE-C encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
+   */
+  public static final String SSE_TYPE_CUSTOM = "custom";
+
+  /**
+   * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not
+   * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64
+   * AES256 symmetric key.
+   */
+  public static final String SSE_KEY = "s3.sse.key";
+
+  /**
+   * If S3 encryption type is SSE-C, input is the base-64 MD5 digest of the secret key. This MD5
+   * must be explicitly passed in by the caller to ensure key integrity.
+   */
+  public static final String SSE_MD5 = "s3.sse.md5";
+
+  /**
+   * Number of threads to use for uploading parts to S3 (shared pool across all output streams),
+   * default to {@link Runtime#availableProcessors()}
+   */
+  public static final String MULTIPART_UPLOAD_THREADS = "s3.multipart.num-threads";
+
+  /**
+   * The size of a single part for multipart upload requests in bytes (default: 32MB). based on S3
+   * requirement, the part size must be at least 5MB. Too ensure performance of the reader and

Review Comment:
   nit: typo in `Too`



##########
aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java:
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.s3;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class TestS3FileIOProperties {

Review Comment:
   Could you please add the below diff to the `iceberg-aws` module? This is actually required to run JUnit5 tests via Gradle
   ```
    project(':iceberg-aws') {
   +  test {
   +    useJUnitPlatform()
   +  }
      dependencies {
      implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
      api project(':iceberg-api')
   ```



##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java:
##########
@@ -0,0 +1,653 @@
+/*
+ * 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.s3;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.aws.glue.GlueCatalog;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.PropertyUtil;
+import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
+import software.amazon.awssdk.services.s3.model.Tag;
+
+public class S3FileIOProperties implements Serializable {
+
+  /**
+   * Type of S3 Server side encryption used, default to {@link S3FileIOProperties#SSE_TYPE_NONE}.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   */
+  public static final String SSE_TYPE = "s3.sse.type";
+
+  /** No server side encryption. */
+  public static final String SSE_TYPE_NONE = "none";
+
+  /**
+   * S3 SSE-KMS encryption.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
+   */
+  public static final String SSE_TYPE_KMS = "kms";
+
+  /**
+   * S3 SSE-S3 encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
+   */
+  public static final String SSE_TYPE_S3 = "s3";
+
+  /**
+   * S3 SSE-C encryption.
+   *
+   * <p>For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
+   */
+  public static final String SSE_TYPE_CUSTOM = "custom";
+
+  /**
+   * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not
+   * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64
+   * AES256 symmetric key.
+   */
+  public static final String SSE_KEY = "s3.sse.key";
+
+  /**
+   * If S3 encryption type is SSE-C, input is the base-64 MD5 digest of the secret key. This MD5
+   * must be explicitly passed in by the caller to ensure key integrity.
+   */
+  public static final String SSE_MD5 = "s3.sse.md5";
+
+  /**
+   * Number of threads to use for uploading parts to S3 (shared pool across all output streams),
+   * default to {@link Runtime#availableProcessors()}
+   */
+  public static final String MULTIPART_UPLOAD_THREADS = "s3.multipart.num-threads";
+
+  /**
+   * The size of a single part for multipart upload requests in bytes (default: 32MB). based on S3
+   * requirement, the part size must be at least 5MB. Too ensure performance of the reader and
+   * writer, the part size must be less than 2GB.
+   *
+   * <p>For more details, see https://docs.aws.amazon.com/AmazonS3/latest/dev/qfacts.html
+   */
+  public static final String MULTIPART_SIZE = "s3.multipart.part-size-bytes";
+
+  public static final int MULTIPART_SIZE_DEFAULT = 32 * 1024 * 1024;
+  public static final int MULTIPART_SIZE_MIN = 5 * 1024 * 1024;
+
+  /**
+   * The threshold expressed as a factor times the multipart size at which to switch from uploading
+   * using a single put object request to uploading using multipart upload (default: 1.5).
+   */
+  public static final String MULTIPART_THRESHOLD_FACTOR = "s3.multipart.threshold";
+
+  public static final double MULTIPART_THRESHOLD_FACTOR_DEFAULT = 1.5;
+
+  /**
+   * Location to put staging files for upload to S3, default to temp directory set in
+   * java.io.tmpdir.
+   */
+  public static final String STAGING_DIRECTORY = "s3.staging-dir";
+
+  /**
+   * Used to configure canned access control list (ACL) for S3 client to use during write. If not
+   * set, ACL will not be set for requests.
+   *
+   * <p>The input must be one of {@link software.amazon.awssdk.services.s3.model.ObjectCannedACL},
+   * such as 'public-read-write' For more details:
+   * https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html
+   */
+  public static final String ACL = "s3.acl";
+
+  /**
+   * Configure an alternative endpoint of the S3 service for S3FileIO to access.
+   *
+   * <p>This could be used to use S3FileIO with any s3-compatible object storage service that has a
+   * different endpoint, or access a private S3 endpoint in a virtual private cloud.
+   */
+  public static final String ENDPOINT = "s3.endpoint";
+
+  /**
+   * If set {@code true}, requests to S3FileIO will use Path-Style, otherwise, Virtual Hosted-Style
+   * will be used.
+   *
+   * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/userguide/VirtualHosting.html
+   */
+  public static final String PATH_STYLE_ACCESS = "s3.path-style-access";
+
+  public static final boolean PATH_STYLE_ACCESS_DEFAULT = false;
+
+  /**
+   * Configure the static access key ID used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #SESSION_TOKEN} is set, session credential is used, otherwise basic credential is used.
+   */
+  public static final String ACCESS_KEY_ID = "s3.access-key-id";
+
+  /**
+   * Configure the static secret access key used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the basic or session credentials provided
+   * instead of reading the default credential chain to create S3 access credentials. If {@link
+   * #SESSION_TOKEN} is set, session credential is used, otherwise basic credential is used.
+   */
+  public static final String SECRET_ACCESS_KEY = "s3.secret-access-key";
+
+  /**
+   * Configure the static session token used to access S3FileIO.
+   *
+   * <p>When set, the default client factory will use the session credentials provided instead of
+   * reading the default credential chain to create S3 access credentials.
+   */
+  public static final String SESSION_TOKEN = "s3.session-token";
+
+  /**
+   * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an
+   * access point.
+   *
+   * <p>By default, attempting to use an access point in a different region will throw an exception.
+   * When enabled, this property allows using access points in other regions.
+   *
+   * <p>For more details see:
+   * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/services/s3/S3Configuration.html#useArnRegionEnabled--
+   */
+  public static final String USE_ARN_REGION_ENABLED = "s3.use-arn-region-enabled";
+
+  public static final boolean USE_ARN_REGION_ENABLED_DEFAULT = false;
+
+  /** Enables eTag checks for S3 PUT and MULTIPART upload requests. */
+  public static final String CHECKSUM_ENABLED = "s3.checksum-enabled";
+
+  public static final boolean CHECKSUM_ENABLED_DEFAULT = false;
+
+  public static final String REMOTE_SIGNING_ENABLED = "s3.remote-signing-enabled";
+
+  public static final boolean REMOTE_SIGNING_ENABLED_DEFAULT = false;
+
+  /** Configure the batch size used when deleting multiple files from a given S3 bucket */
+  public static final String DELETE_BATCH_SIZE = "s3.delete.batch-size";
+
+  /**
+   * Default batch size used when deleting files.
+   *
+   * <p>Refer to https://github.com/apache/hadoop/commit/56dee667707926f3796c7757be1a133a362f05c9
+   * for more details on why this value was chosen.
+   */
+  public static final int DELETE_BATCH_SIZE_DEFAULT = 250;
+
+  /**
+   * Max possible batch size for deletion. Currently, a max of 1000 keys can be deleted in one
+   * batch. https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
+   */
+  public static final int DELETE_BATCH_SIZE_MAX = 1000;
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when writing. To set, we can pass a catalog property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.tags.my_key=my_val
+   */
+  public static final String WRITE_TAGS_PREFIX = "s3.write.tags.";
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.table-tag-enabled=true
+   */
+  public static final String WRITE_TABLE_TAG_ENABLED = "s3.write.table-tag-enabled";
+
+  public static final boolean WRITE_TABLE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link GlueCatalog} to tag objects when writing. To set, we can pass a catalog
+   * property.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-tagging.html
+   *
+   * <p>Example: s3.write.namespace-tag-enabled=true
+   */
+  public static final String WRITE_NAMESPACE_TAG_ENABLED = "s3.write.namespace-tag-enabled";
+
+  public static final boolean WRITE_NAMESPACE_TAG_ENABLED_DEFAULT = false;
+
+  /**
+   * Tag name that will be used by {@link #WRITE_TAGS_PREFIX} when {@link #WRITE_TABLE_TAG_ENABLED}
+   * is enabled
+   *
+   * <p>Example: iceberg.table=tableName
+   */
+  public static final String S3_TAG_ICEBERG_TABLE = "iceberg.table";
+
+  /**
+   * Tag name that will be used by {@link #WRITE_TAGS_PREFIX} when {@link
+   * #WRITE_NAMESPACE_TAG_ENABLED} is enabled
+   *
+   * <p>Example: iceberg.namespace=namespaceName
+   */
+  public static final String S3_TAG_ICEBERG_NAMESPACE = "iceberg.namespace";
+
+  /**
+   * Used by {@link S3FileIO} to tag objects when deleting. When this config is set, objects are
+   * tagged with the configured key-value pairs before deletion. This is considered a soft-delete,
+   * because users are able to configure tag-based object lifecycle policy at bucket level to
+   * transition objects to different tiers.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-lifecycle-mgmt.html
+   *
+   * <p>Example: s3.delete.tags.my_key=my_val
+   */
+  public static final String DELETE_TAGS_PREFIX = "s3.delete.tags.";
+
+  /**
+   * Number of threads to use for adding delete tags to S3 objects, default to {@link
+   * Runtime#availableProcessors()}
+   */
+  public static final String DELETE_THREADS = "s3.delete.num-threads";
+
+  /**
+   * Determines if {@link S3FileIO} deletes the object when io.delete() is called, default to true.
+   * Once disabled, users are expected to set tags through {@link #DELETE_TAGS_PREFIX} and manage
+   * deleted files through S3 lifecycle policy.
+   */
+  public static final String DELETE_ENABLED = "s3.delete-enabled";
+
+  public static final boolean DELETE_ENABLED_DEFAULT = true;
+
+  /**
+   * Determines if S3 client will use the Acceleration Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/transfer-acceleration.html
+   */
+  public static final String ACCELERATION_ENABLED = "s3.acceleration-enabled";
+
+  public static final boolean ACCELERATION_ENABLED_DEFAULT = false;
+
+  /**
+   * Determines if S3 client will use the Dualstack Mode, default to false.
+   *
+   * <p>For more details, see
+   * https://docs.aws.amazon.com/AmazonS3/latest/userguide/dual-stack-endpoints.html
+   */
+  public static final String DUALSTACK_ENABLED = "s3.dualstack-enabled";
+
+  public static final boolean DUALSTACK_ENABLED_DEFAULT = false;
+
+  /**
+   * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can
+   * pass a catalog property.
+   *
+   * <p>For more details, see https://aws.amazon.com/s3/features/access-points/
+   *
+   * <p>Example: s3.access-points.my-bucket=access-point
+   */
+  public static final String ACCESS_POINTS_PREFIX = "s3.access-points.";
+
+  /**
+   * This flag controls whether the S3 client will be initialized during the S3FileIO
+   * initialization, instead of default lazy initialization upon use. This is needed for cases that
+   * the credentials to use might change and needs to be preloaded.
+   */
+  public static final String PRELOAD_CLIENT_ENABLED = "s3.preload-client-enabled";
+
+  public static final boolean PRELOAD_CLIENT_ENABLED_DEFAULT = false;
+
+  private String sseType;
+  private String sseKey;
+  private String sseMd5;
+  private String accessKeyId;
+  private String secretAccessKey;
+  private String sessionToken;
+  private int multipartUploadThreads;
+  private int multiPartSize;
+  private int deleteBatchSize;
+  private double multipartThresholdFactor;
+  private String stagingDirectory;
+  private ObjectCannedACL acl;
+  private boolean isChecksumEnabled;
+  private final Set<Tag> writeTags;
+  private boolean isWriteTableTagEnabled;
+  private boolean isWriteNamespaceTagEnabled;
+  private final Set<Tag> deleteTags;
+  private int deleteThreads;
+  private boolean isDeleteEnabled;
+  private final Map<String, String> bucketToAccessPointMapping;
+  private boolean isPreloadClientEnabled;
+  private boolean isDualStackEnabled;
+  private boolean isPathStyleAccess;
+  private boolean isUseArnRegionEnabled;
+  private boolean isAccelerationEnabled;
+  private String endpoint;
+  private final boolean isRemoteSigningEnabled;
+
+  public S3FileIOProperties() {
+    this.sseType = SSE_TYPE_NONE;
+    this.sseKey = null;
+    this.sseMd5 = null;
+    this.accessKeyId = null;
+    this.secretAccessKey = null;
+    this.sessionToken = null;
+    this.acl = null;
+    this.endpoint = null;
+    this.multipartUploadThreads = Runtime.getRuntime().availableProcessors();
+    this.multiPartSize = MULTIPART_SIZE_DEFAULT;
+    this.multipartThresholdFactor = MULTIPART_THRESHOLD_FACTOR_DEFAULT;
+    this.deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT;
+    this.stagingDirectory = System.getProperty("java.io.tmpdir");
+    this.isChecksumEnabled = CHECKSUM_ENABLED_DEFAULT;
+    this.writeTags = Sets.newHashSet();
+    this.isWriteTableTagEnabled = WRITE_TABLE_TAG_ENABLED_DEFAULT;
+    this.isWriteNamespaceTagEnabled = WRITE_NAMESPACE_TAG_ENABLED_DEFAULT;
+    this.deleteTags = Sets.newHashSet();
+    this.deleteThreads = Runtime.getRuntime().availableProcessors();
+    this.isDeleteEnabled = DELETE_ENABLED_DEFAULT;
+    this.bucketToAccessPointMapping = Collections.emptyMap();
+    this.isPreloadClientEnabled = PRELOAD_CLIENT_ENABLED_DEFAULT;
+    this.isDualStackEnabled = DUALSTACK_ENABLED_DEFAULT;
+    this.isPathStyleAccess = PATH_STYLE_ACCESS_DEFAULT;
+    this.isUseArnRegionEnabled = USE_ARN_REGION_ENABLED_DEFAULT;
+    this.isAccelerationEnabled = ACCELERATION_ENABLED_DEFAULT;
+    this.isRemoteSigningEnabled = REMOTE_SIGNING_ENABLED_DEFAULT;
+
+    ValidationException.check(
+        keyIdAccessKeyBothConfigured(),
+        "S3 client access key ID and secret access key must be set at the same time");
+  }
+
+  public S3FileIOProperties(Map<String, String> properties) {
+    this.sseType = properties.getOrDefault(SSE_TYPE, SSE_TYPE_NONE);
+    this.sseKey = properties.get(SSE_KEY);
+    this.sseMd5 = properties.get(SSE_MD5);
+    this.accessKeyId = properties.get(ACCESS_KEY_ID);
+    this.secretAccessKey = properties.get(SECRET_ACCESS_KEY);
+    this.sessionToken = properties.get(SESSION_TOKEN);
+    if (SSE_TYPE_CUSTOM.equals(sseType)) {
+      Preconditions.checkNotNull(

Review Comment:
   @jackye1995 thoughts on switching this to using `Preconditions.checkArgument(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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #7505: Move all S3FileIO related properties into a separate class S3FileIOProperties

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


##########
aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java:
##########
@@ -68,47 +68,63 @@ public class AwsProperties implements Serializable {
    * AwsProperties#S3FILEIO_SSE_TYPE_NONE}.
    *
    * <p>For more details: https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html
+   *
+   * @deprecated use {@link org.apache.iceberg.aws.s3.S3FileIOProperties} instead

Review Comment:
   Let's first say 1.4 for now, so we at least have those changes available



-- 
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