This is an automated email from the ASF dual-hosted git repository.

amoghj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 3ab00171b4 Remove usages of S3 fields from AwsProperties within s3 
package (#7534)
3ab00171b4 is described below

commit 3ab00171b48bb793a3b71845eb12d5077ba892f1
Author: akshayakp97 <[email protected]>
AuthorDate: Fri May 5 14:05:52 2023 -0700

    Remove usages of S3 fields from AwsProperties within s3 package (#7534)
    
    Remove usages of S3 fields from AwsProperties within s3 package
---
 .../iceberg/aws/s3/TestS3FileIOIntegration.java    | 52 ++++++++++-----------
 .../iceberg/aws/s3/TestS3MultipartUpload.java      | 23 +++++----
 .../java/org/apache/iceberg/aws/s3/BaseS3File.java | 14 +++---
 .../java/org/apache/iceberg/aws/s3/S3FileIO.java   | 52 ++++++++++-----------
 .../org/apache/iceberg/aws/s3/S3InputFile.java     | 22 +++++----
 .../org/apache/iceberg/aws/s3/S3InputStream.java   | 14 +++---
 .../org/apache/iceberg/aws/s3/S3OutputFile.java    | 19 ++++----
 .../org/apache/iceberg/aws/s3/S3OutputStream.java  | 30 ++++++------
 .../org/apache/iceberg/aws/s3/S3RequestUtil.java   | 54 +++++++++++-----------
 .../apache/iceberg/aws/TestAwsClientFactories.java |  7 +--
 .../apache/iceberg/aws/s3/TestS3OutputStream.java  | 27 ++++++-----
 .../apache/iceberg/aws/s3/TestS3RequestUtil.java   | 33 +++++++------
 12 files changed, 174 insertions(+), 173 deletions(-)

diff --git 
a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
 
b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
index d5588d67e0..b93fdbf8f6 100644
--- 
a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
+++ 
b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java
@@ -149,14 +149,14 @@ public class TestS3FileIOIntegration {
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3);
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testRegion(), 
accessPointName)));
     validateRead(s3FileIO);
   }
 
   @Test
   public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception {
-    
clientFactory.initialize(ImmutableMap.of(AwsProperties.S3_USE_ARN_REGION_ENABLED,
 "true"));
+    
clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED,
 "true"));
     S3Client s3Client = clientFactory.s3();
     s3Client.putObject(
         PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(),
@@ -172,7 +172,7 @@ public class TestS3FileIOIntegration {
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3);
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), 
crossRegionAccessPointName)));
     validateRead(s3FileIO);
   }
@@ -193,7 +193,7 @@ public class TestS3FileIOIntegration {
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3);
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testRegion(), 
accessPointName)));
     write(s3FileIO);
     InputStream stream =
@@ -205,12 +205,12 @@ public class TestS3FileIOIntegration {
 
   @Test
   public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception 
{
-    
clientFactory.initialize(ImmutableMap.of(AwsProperties.S3_USE_ARN_REGION_ENABLED,
 "true"));
+    
clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED,
 "true"));
     S3Client s3Client = clientFactory.s3();
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3);
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), 
crossRegionAccessPointName)));
     write(s3FileIO);
     InputStream stream =
@@ -228,8 +228,8 @@ public class TestS3FileIOIntegration {
 
   @Test
   public void testServerSideS3Encryption() throws Exception {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_S3);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setSseType(S3FileIOProperties.SSE_TYPE_S3);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     write(s3FileIO);
     validateRead(s3FileIO);
@@ -241,9 +241,9 @@ public class TestS3FileIOIntegration {
 
   @Test
   public void testServerSideKmsEncryption() throws Exception {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_KMS);
-    properties.setS3FileIoSseKey(kmsKeyArn);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS);
+    properties.setSseKey(kmsKeyArn);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     write(s3FileIO);
     validateRead(s3FileIO);
@@ -256,8 +256,8 @@ public class TestS3FileIOIntegration {
 
   @Test
   public void testServerSideKmsEncryptionWithDefaultKey() throws Exception {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_KMS);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     write(s3FileIO);
     validateRead(s3FileIO);
@@ -285,10 +285,10 @@ public class TestS3FileIOIntegration {
     String md5 =
         new String(encoder.encode(digest.digest(secretKey.getEncoded())), 
StandardCharsets.UTF_8);
 
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_CUSTOM);
-    properties.setS3FileIoSseKey(encodedKey);
-    properties.setS3FileIoSseMd5(md5);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setSseType(S3FileIOProperties.SSE_TYPE_CUSTOM);
+    properties.setSseKey(encodedKey);
+    properties.setSseMd5(md5);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     write(s3FileIO);
     validateRead(s3FileIO);
@@ -309,8 +309,8 @@ public class TestS3FileIOIntegration {
 
   @Test
   public void testACL() throws Exception {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoAcl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setAcl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     write(s3FileIO);
     validateRead(s3FileIO);
@@ -341,7 +341,7 @@ public class TestS3FileIOIntegration {
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, 
getDeletionTestProperties());
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testRegion(), 
accessPointName)));
     testDeleteFiles(deletionBatchSize * 2, s3FileIO);
   }
@@ -352,7 +352,7 @@ public class TestS3FileIOIntegration {
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, 
getDeletionTestProperties());
     s3FileIO.initialize(
         ImmutableMap.of(
-            AwsProperties.S3_ACCESS_POINTS_PREFIX + bucketName,
+            S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName,
             testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), 
crossRegionAccessPointName)));
     testDeleteFiles(deletionBatchSize * 2, s3FileIO);
   }
@@ -392,8 +392,8 @@ public class TestS3FileIOIntegration {
   @SuppressWarnings("DangerousParallelStreamUsage")
   @Test
   public void testPrefixDelete() {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoDeleteBatchSize(100);
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setDeleteBatchSize(100);
     S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties);
     String deletePrefix = String.format("s3://%s/%s/%s", bucketName, prefix, 
"prefix-delete-test");
 
@@ -409,9 +409,9 @@ public class TestS3FileIOIntegration {
             });
   }
 
-  private AwsProperties getDeletionTestProperties() {
-    AwsProperties properties = new AwsProperties();
-    properties.setS3FileIoDeleteBatchSize(deletionBatchSize);
+  private S3FileIOProperties getDeletionTestProperties() {
+    S3FileIOProperties properties = new S3FileIOProperties();
+    properties.setDeleteBatchSize(deletionBatchSize);
     return properties;
   }
 
diff --git 
a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java 
b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
index 9cb3feadf9..3ebc51aab8 100644
--- 
a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
+++ 
b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
@@ -25,7 +25,6 @@ import java.util.function.Supplier;
 import java.util.stream.IntStream;
 import org.apache.iceberg.aws.AwsClientFactories;
 import org.apache.iceberg.aws.AwsIntegTestUtil;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.io.PositionOutputStream;
 import org.apache.iceberg.io.SeekableInputStream;
 import org.junit.AfterClass;
@@ -42,7 +41,7 @@ public class TestS3MultipartUpload {
   private static S3Client s3;
   private static String bucketName;
   private static String prefix;
-  private static AwsProperties properties;
+  private static S3FileIOProperties properties;
   private static S3FileIO io;
   private String objectUri;
 
@@ -51,9 +50,9 @@ public class TestS3MultipartUpload {
     s3 = AwsClientFactories.defaultFactory().s3();
     bucketName = AwsIntegTestUtil.testBucketName();
     prefix = UUID.randomUUID().toString();
-    properties = new AwsProperties();
-    
properties.setS3FileIoMultiPartSize(AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN);
-    properties.setS3ChecksumEnabled(true);
+    properties = new S3FileIOProperties();
+    properties.setMultiPartSize(S3FileIOProperties.MULTIPART_SIZE_MIN);
+    properties.setChecksumEnabled(true);
     io = new S3FileIO(() -> s3, properties);
   }
 
@@ -73,14 +72,14 @@ public class TestS3MultipartUpload {
     int parts = 200;
     writeInts(objectUri, parts, random::nextInt);
     Assert.assertEquals(
-        parts * (long) AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN,
+        parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN,
         io.newInputFile(objectUri).getLength());
   }
 
   @Test
   public void testManyPartsWriteWithBytes() throws IOException {
     int parts = 200;
-    byte[] bytes = new byte[AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN];
+    byte[] bytes = new byte[S3FileIOProperties.MULTIPART_SIZE_MIN];
     writeBytes(
         objectUri,
         parts,
@@ -89,7 +88,7 @@ public class TestS3MultipartUpload {
           return bytes;
         });
     Assert.assertEquals(
-        parts * (long) AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN,
+        parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN,
         io.newInputFile(objectUri).getLength());
   }
 
@@ -101,8 +100,8 @@ public class TestS3MultipartUpload {
 
   @Test
   public void testContentsWriteWithBytes() throws IOException {
-    byte[] bytes = new byte[AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN];
-    for (int i = 0; i < AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN; i++) {
+    byte[] bytes = new byte[S3FileIOProperties.MULTIPART_SIZE_MIN];
+    for (int i = 0; i < S3FileIOProperties.MULTIPART_SIZE_MIN; i++) {
       bytes[i] = 6;
     }
     writeBytes(objectUri, 10, () -> bytes);
@@ -111,7 +110,7 @@ public class TestS3MultipartUpload {
 
   @Test
   public void testUploadRemainder() throws IOException {
-    long length = 3 * AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN + 2 * 1024 * 
1024;
+    long length = 3 * S3FileIOProperties.MULTIPART_SIZE_MIN + 2 * 1024 * 1024;
     writeInts(objectUri, 1, length, random::nextInt);
     Assert.assertEquals(length, io.newInputFile(objectUri).getLength());
   }
@@ -128,7 +127,7 @@ public class TestS3MultipartUpload {
   }
 
   private void writeInts(String fileUri, int parts, Supplier<Integer> writer) {
-    writeInts(fileUri, parts, AwsProperties.S3FILEIO_MULTIPART_SIZE_MIN, 
writer);
+    writeInts(fileUri, parts, S3FileIOProperties.MULTIPART_SIZE_MIN, writer);
   }
 
   private void writeInts(String fileUri, int parts, long partSize, 
Supplier<Integer> writer) {
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/BaseS3File.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/BaseS3File.java
index 24b2cf2615..88b212a37b 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/BaseS3File.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/BaseS3File.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iceberg.aws.s3;
 
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.metrics.MetricsContext;
 import software.amazon.awssdk.http.HttpStatusCode;
 import software.amazon.awssdk.services.s3.S3Client;
@@ -29,14 +28,15 @@ import software.amazon.awssdk.services.s3.model.S3Exception;
 abstract class BaseS3File {
   private final S3Client client;
   private final S3URI uri;
-  private final AwsProperties awsProperties;
+  private final S3FileIOProperties s3FileIOProperties;
   private HeadObjectResponse metadata;
   private final MetricsContext metrics;
 
-  BaseS3File(S3Client client, S3URI uri, AwsProperties awsProperties, 
MetricsContext metrics) {
+  BaseS3File(
+      S3Client client, S3URI uri, S3FileIOProperties s3FileIOProperties, 
MetricsContext metrics) {
     this.client = client;
     this.uri = uri;
-    this.awsProperties = awsProperties;
+    this.s3FileIOProperties = s3FileIOProperties;
     this.metrics = metrics;
   }
 
@@ -52,8 +52,8 @@ abstract class BaseS3File {
     return uri;
   }
 
-  public AwsProperties awsProperties() {
-    return awsProperties;
+  public S3FileIOProperties s3FileIOProperties() {
+    return s3FileIOProperties;
   }
 
   protected MetricsContext metrics() {
@@ -81,7 +81,7 @@ abstract class BaseS3File {
     if (metadata == null) {
       HeadObjectRequest.Builder requestBuilder =
           HeadObjectRequest.builder().bucket(uri().bucket()).key(uri().key());
-      S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
+      S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder);
       metadata = client().headObject(requestBuilder.build());
     }
 
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
index 7bf9ef1421..1784cd975f 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
@@ -30,7 +30,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 import org.apache.iceberg.aws.AwsClientFactories;
 import org.apache.iceberg.aws.AwsClientFactory;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.common.DynConstructors;
 import org.apache.iceberg.io.BulkDeletionFailureException;
 import org.apache.iceberg.io.CredentialSupplier;
@@ -84,7 +83,7 @@ public class S3FileIO
 
   private String credential = null;
   private SerializableSupplier<S3Client> s3;
-  private AwsProperties awsProperties;
+  private S3FileIOProperties s3FileIOProperties;
   private SerializableMap<String, String> properties = null;
   private transient volatile S3Client client;
   private MetricsContext metrics = MetricsContext.nullMetrics();
@@ -101,60 +100,59 @@ public class S3FileIO
   }
 
   /**
-   * Constructor with custom s3 supplier and default AWS properties.
+   * Constructor with custom s3 supplier and S3FileIO properties.
    *
    * <p>Calling {@link S3FileIO#initialize(Map)} will overwrite information 
set in this constructor.
    *
    * @param s3 s3 supplier
    */
   public S3FileIO(SerializableSupplier<S3Client> s3) {
-    this(s3, new AwsProperties());
+    this(s3, new S3FileIOProperties());
   }
 
   /**
-   * Constructor with custom s3 supplier and AWS properties.
+   * Constructor with custom s3 supplier and S3FileIO properties.
    *
    * <p>Calling {@link S3FileIO#initialize(Map)} will overwrite information 
set in this constructor.
    *
    * @param s3 s3 supplier
-   * @param awsProperties aws properties
+   * @param s3FileIOProperties S3 FileIO properties
    */
-  public S3FileIO(SerializableSupplier<S3Client> s3, AwsProperties 
awsProperties) {
+  public S3FileIO(SerializableSupplier<S3Client> s3, S3FileIOProperties 
s3FileIOProperties) {
     this.s3 = s3;
-    this.awsProperties = awsProperties;
     this.createStack = Thread.currentThread().getStackTrace();
   }
 
   @Override
   public InputFile newInputFile(String path) {
-    return S3InputFile.fromLocation(path, client(), awsProperties, metrics);
+    return S3InputFile.fromLocation(path, client(), s3FileIOProperties, 
metrics);
   }
 
   @Override
   public InputFile newInputFile(String path, long length) {
-    return S3InputFile.fromLocation(path, length, client(), awsProperties, 
metrics);
+    return S3InputFile.fromLocation(path, length, client(), 
s3FileIOProperties, metrics);
   }
 
   @Override
   public OutputFile newOutputFile(String path) {
-    return S3OutputFile.fromLocation(path, client(), awsProperties, metrics);
+    return S3OutputFile.fromLocation(path, client(), s3FileIOProperties, 
metrics);
   }
 
   @Override
   public void deleteFile(String path) {
-    if (awsProperties.s3DeleteTags() != null && 
!awsProperties.s3DeleteTags().isEmpty()) {
+    if (s3FileIOProperties.deleteTags() != null && 
!s3FileIOProperties.deleteTags().isEmpty()) {
       try {
-        tagFileToDelete(path, awsProperties.s3DeleteTags());
+        tagFileToDelete(path, s3FileIOProperties.deleteTags());
       } catch (S3Exception e) {
-        LOG.warn("Failed to add delete tags: {} to {}", 
awsProperties.s3DeleteTags(), path, e);
+        LOG.warn("Failed to add delete tags: {} to {}", 
s3FileIOProperties.deleteTags(), path, e);
       }
     }
 
-    if (!awsProperties.isS3DeleteEnabled()) {
+    if (!s3FileIOProperties.isDeleteEnabled()) {
       return;
     }
 
-    S3URI location = new S3URI(path, 
awsProperties.s3BucketToAccessPointMapping());
+    S3URI location = new S3URI(path, 
s3FileIOProperties.bucketToAccessPointMapping());
     DeleteObjectRequest deleteRequest =
         
DeleteObjectRequest.builder().bucket(location.bucket()).key(location.key()).build();
 
@@ -176,7 +174,7 @@ public class S3FileIO
    */
   @Override
   public void deleteFiles(Iterable<String> paths) throws 
BulkDeletionFailureException {
-    if (awsProperties.s3DeleteTags() != null && 
!awsProperties.s3DeleteTags().isEmpty()) {
+    if (s3FileIOProperties.deleteTags() != null && 
!s3FileIOProperties.deleteTags().isEmpty()) {
       Tasks.foreach(paths)
           .noRetry()
           .executeWith(executorService())
@@ -185,22 +183,22 @@ public class S3FileIO
               (path, exc) ->
                   LOG.warn(
                       "Failed to add delete tags: {} to {}",
-                      awsProperties.s3DeleteTags(),
+                      s3FileIOProperties.deleteTags(),
                       path,
                       exc))
-          .run(path -> tagFileToDelete(path, awsProperties.s3DeleteTags()));
+          .run(path -> tagFileToDelete(path, s3FileIOProperties.deleteTags()));
     }
 
-    if (awsProperties.isS3DeleteEnabled()) {
+    if (s3FileIOProperties.isDeleteEnabled()) {
       SetMultimap<String, String> bucketToObjects =
           Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet);
       List<Future<List<String>>> deletionTasks = Lists.newArrayList();
       for (String path : paths) {
-        S3URI location = new S3URI(path, 
awsProperties.s3BucketToAccessPointMapping());
+        S3URI location = new S3URI(path, 
s3FileIOProperties.bucketToAccessPointMapping());
         String bucket = location.bucket();
         String objectKey = location.key();
         bucketToObjects.get(bucket).add(objectKey);
-        if (bucketToObjects.get(bucket).size() == 
awsProperties.s3FileIoDeleteBatchSize()) {
+        if (bucketToObjects.get(bucket).size() == 
s3FileIOProperties.deleteBatchSize()) {
           Set<String> keys = Sets.newHashSet(bucketToObjects.get(bucket));
           Future<List<String>> deletionTask =
               executorService().submit(() -> deleteBatch(bucket, keys));
@@ -242,7 +240,7 @@ public class S3FileIO
   }
 
   private void tagFileToDelete(String path, Set<Tag> deleteTags) throws 
S3Exception {
-    S3URI location = new S3URI(path, 
awsProperties.s3BucketToAccessPointMapping());
+    S3URI location = new S3URI(path, 
s3FileIOProperties.bucketToAccessPointMapping());
     String bucket = location.bucket();
     String objectKey = location.key();
     GetObjectTaggingRequest getObjectTaggingRequest =
@@ -296,7 +294,7 @@ public class S3FileIO
 
   @Override
   public Iterable<FileInfo> listPrefix(String prefix) {
-    S3URI s3uri = new S3URI(prefix, 
awsProperties.s3BucketToAccessPointMapping());
+    S3URI s3uri = new S3URI(prefix, 
s3FileIOProperties.bucketToAccessPointMapping());
     ListObjectsV2Request request =
         
ListObjectsV2Request.builder().bucket(s3uri.bucket()).prefix(s3uri.key()).build();
 
@@ -342,7 +340,7 @@ public class S3FileIO
         if (executorService == null) {
           executorService =
               ThreadPools.newWorkerPool(
-                  "iceberg-s3fileio-delete", 
awsProperties.s3FileIoDeleteThreads());
+                  "iceberg-s3fileio-delete", 
s3FileIOProperties.deleteThreads());
         }
       }
     }
@@ -358,7 +356,7 @@ public class S3FileIO
   @Override
   public void initialize(Map<String, String> props) {
     this.properties = SerializableMap.copyOf(props);
-    this.awsProperties = new AwsProperties(properties);
+    this.s3FileIOProperties = new S3FileIOProperties(properties);
 
     // Do not override s3 client if it was provided
     if (s3 == null) {
@@ -367,7 +365,7 @@ public class S3FileIO
         this.credential = ((CredentialSupplier) clientFactory).getCredential();
       }
       this.s3 = clientFactory::s3;
-      if (awsProperties.s3PreloadClientEnabled()) {
+      if (s3FileIOProperties.isPreloadClientEnabled()) {
         client();
       }
     }
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java
index aeef04a6b9..5ad82c153f 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iceberg.aws.s3;
 
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.encryption.NativeFileCryptoParameters;
 import org.apache.iceberg.encryption.NativelyEncryptedFile;
 import org.apache.iceberg.io.InputFile;
@@ -31,12 +30,15 @@ public class S3InputFile extends BaseS3File implements 
InputFile, NativelyEncryp
   private Long length;
 
   public static S3InputFile fromLocation(
-      String location, S3Client client, AwsProperties awsProperties, 
MetricsContext metrics) {
+      String location,
+      S3Client client,
+      S3FileIOProperties s3FileIOProperties,
+      MetricsContext metrics) {
     return new S3InputFile(
         client,
-        new S3URI(location, awsProperties.s3BucketToAccessPointMapping()),
+        new S3URI(location, s3FileIOProperties.bucketToAccessPointMapping()),
         null,
-        awsProperties,
+        s3FileIOProperties,
         metrics);
   }
 
@@ -44,13 +46,13 @@ public class S3InputFile extends BaseS3File implements 
InputFile, NativelyEncryp
       String location,
       long length,
       S3Client client,
-      AwsProperties awsProperties,
+      S3FileIOProperties s3FileIOProperties,
       MetricsContext metrics) {
     return new S3InputFile(
         client,
-        new S3URI(location, awsProperties.s3BucketToAccessPointMapping()),
+        new S3URI(location, s3FileIOProperties.bucketToAccessPointMapping()),
         length > 0 ? length : null,
-        awsProperties,
+        s3FileIOProperties,
         metrics);
   }
 
@@ -58,9 +60,9 @@ public class S3InputFile extends BaseS3File implements 
InputFile, NativelyEncryp
       S3Client client,
       S3URI uri,
       Long length,
-      AwsProperties awsProperties,
+      S3FileIOProperties s3FileIOProperties,
       MetricsContext metrics) {
-    super(client, uri, awsProperties, metrics);
+    super(client, uri, s3FileIOProperties, metrics);
     this.length = length;
   }
 
@@ -80,7 +82,7 @@ public class S3InputFile extends BaseS3File implements 
InputFile, NativelyEncryp
 
   @Override
   public SeekableInputStream newStream() {
-    return new S3InputStream(client(), uri(), awsProperties(), metrics());
+    return new S3InputStream(client(), uri(), s3FileIOProperties(), metrics());
   }
 
   @Override
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java
index 7d83cea3f1..f1d6c30a27 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java
@@ -21,7 +21,6 @@ package org.apache.iceberg.aws.s3;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.exceptions.NotFoundException;
 import org.apache.iceberg.io.FileIOMetricsContext;
 import org.apache.iceberg.io.IOUtil;
@@ -47,7 +46,7 @@ class S3InputStream extends SeekableInputStream implements 
RangeReadable {
   private final StackTraceElement[] createStack;
   private final S3Client s3;
   private final S3URI location;
-  private final AwsProperties awsProperties;
+  private final S3FileIOProperties s3FileIOProperties;
 
   private InputStream stream;
   private long pos = 0;
@@ -60,13 +59,14 @@ class S3InputStream extends SeekableInputStream implements 
RangeReadable {
   private int skipSize = 1024 * 1024;
 
   S3InputStream(S3Client s3, S3URI location) {
-    this(s3, location, new AwsProperties(), MetricsContext.nullMetrics());
+    this(s3, location, new S3FileIOProperties(), MetricsContext.nullMetrics());
   }
 
-  S3InputStream(S3Client s3, S3URI location, AwsProperties awsProperties, 
MetricsContext metrics) {
+  S3InputStream(
+      S3Client s3, S3URI location, S3FileIOProperties s3FileIOProperties, 
MetricsContext metrics) {
     this.s3 = s3;
     this.location = location;
-    this.awsProperties = awsProperties;
+    this.s3FileIOProperties = s3FileIOProperties;
 
     this.readBytes = metrics.counter(FileIOMetricsContext.READ_BYTES, 
Unit.BYTES);
     this.readOperations = 
metrics.counter(FileIOMetricsContext.READ_OPERATIONS);
@@ -137,7 +137,7 @@ class S3InputStream extends SeekableInputStream implements 
RangeReadable {
     GetObjectRequest.Builder requestBuilder =
         
GetObjectRequest.builder().bucket(location.bucket()).key(location.key()).range(range);
 
-    S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
+    S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder);
 
     return s3.getObject(requestBuilder.build(), 
ResponseTransformer.toInputStream());
   }
@@ -184,7 +184,7 @@ class S3InputStream extends SeekableInputStream implements 
RangeReadable {
             .key(location.key())
             .range(String.format("bytes=%s-", pos));
 
-    S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
+    S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder);
 
     closeStream();
 
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java
index 5345961315..abe7a55fba 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java
@@ -20,7 +20,6 @@ package org.apache.iceberg.aws.s3;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.encryption.NativeFileCryptoParameters;
 import org.apache.iceberg.encryption.NativelyEncryptedFile;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
@@ -34,16 +33,20 @@ public class S3OutputFile extends BaseS3File implements 
OutputFile, NativelyEncr
   private NativeFileCryptoParameters nativeEncryptionParameters;
 
   public static S3OutputFile fromLocation(
-      String location, S3Client client, AwsProperties awsProperties, 
MetricsContext metrics) {
+      String location,
+      S3Client client,
+      S3FileIOProperties s3FileIOProperties,
+      MetricsContext metrics) {
     return new S3OutputFile(
         client,
-        new S3URI(location, awsProperties.s3BucketToAccessPointMapping()),
-        awsProperties,
+        new S3URI(location, s3FileIOProperties.bucketToAccessPointMapping()),
+        s3FileIOProperties,
         metrics);
   }
 
-  S3OutputFile(S3Client client, S3URI uri, AwsProperties awsProperties, 
MetricsContext metrics) {
-    super(client, uri, awsProperties, metrics);
+  S3OutputFile(
+      S3Client client, S3URI uri, S3FileIOProperties s3FileIOProperties, 
MetricsContext metrics) {
+    super(client, uri, s3FileIOProperties, metrics);
   }
 
   /**
@@ -64,7 +67,7 @@ public class S3OutputFile extends BaseS3File implements 
OutputFile, NativelyEncr
   @Override
   public PositionOutputStream createOrOverwrite() {
     try {
-      return new S3OutputStream(client(), uri(), awsProperties(), metrics());
+      return new S3OutputStream(client(), uri(), s3FileIOProperties(), 
metrics());
     } catch (IOException e) {
       throw new UncheckedIOException("Failed to create output stream for 
location: " + uri(), e);
     }
@@ -72,7 +75,7 @@ public class S3OutputFile extends BaseS3File implements 
OutputFile, NativelyEncr
 
   @Override
   public InputFile toInputFile() {
-    return new S3InputFile(client(), uri(), null, awsProperties(), metrics());
+    return new S3InputFile(client(), uri(), null, s3FileIOProperties(), 
metrics());
   }
 
   @Override
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
index 5e58be1609..dfc7aaa460 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
@@ -43,7 +43,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.stream.Collectors;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.io.FileIOMetricsContext;
 import org.apache.iceberg.io.PositionOutputStream;
 import org.apache.iceberg.metrics.Counter;
@@ -85,7 +84,7 @@ class S3OutputStream extends PositionOutputStream {
   private final StackTraceElement[] createStack;
   private final S3Client s3;
   private final S3URI location;
-  private final AwsProperties awsProperties;
+  private final S3FileIOProperties s3FileIOProperties;
   private final Set<Tag> writeTags;
 
   private CountingOutputStream stream;
@@ -107,7 +106,8 @@ class S3OutputStream extends PositionOutputStream {
   private boolean closed = false;
 
   @SuppressWarnings("StaticAssignmentInConstructor")
-  S3OutputStream(S3Client s3, S3URI location, AwsProperties awsProperties, 
MetricsContext metrics)
+  S3OutputStream(
+      S3Client s3, S3URI location, S3FileIOProperties s3FileIOProperties, 
MetricsContext metrics)
       throws IOException {
     if (executorService == null) {
       synchronized (S3OutputStream.class) {
@@ -116,7 +116,7 @@ class S3OutputStream extends PositionOutputStream {
               MoreExecutors.getExitingExecutorService(
                   (ThreadPoolExecutor)
                       Executors.newFixedThreadPool(
-                          awsProperties.s3FileIoMultipartUploadThreads(),
+                          s3FileIOProperties.multipartUploadThreads(),
                           new ThreadFactoryBuilder()
                               .setDaemon(true)
                               .setNameFormat("iceberg-s3fileio-upload-%d")
@@ -127,16 +127,16 @@ class S3OutputStream extends PositionOutputStream {
 
     this.s3 = s3;
     this.location = location;
-    this.awsProperties = awsProperties;
-    this.writeTags = awsProperties.s3WriteTags();
+    this.s3FileIOProperties = s3FileIOProperties;
+    this.writeTags = s3FileIOProperties.writeTags();
 
     this.createStack = Thread.currentThread().getStackTrace();
 
-    this.multiPartSize = awsProperties.s3FileIoMultiPartSize();
+    this.multiPartSize = s3FileIOProperties.multiPartSize();
     this.multiPartThresholdSize =
-        (int) (multiPartSize * 
awsProperties.s3FileIOMultipartThresholdFactor());
-    this.stagingDirectory = new File(awsProperties.s3fileIoStagingDirectory());
-    this.isChecksumEnabled = awsProperties.isS3ChecksumEnabled();
+        (int) (multiPartSize * s3FileIOProperties.multipartThresholdFactor());
+    this.stagingDirectory = new File(s3FileIOProperties.stagingDirectory());
+    this.isChecksumEnabled = s3FileIOProperties.isChecksumEnabled();
     try {
       this.completeMessageDigest =
           isChecksumEnabled ? MessageDigest.getInstance(digestAlgorithm) : 
null;
@@ -279,8 +279,8 @@ class S3OutputStream extends PositionOutputStream {
       requestBuilder.tagging(Tagging.builder().tagSet(writeTags).build());
     }
 
-    S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
-    S3RequestUtil.configurePermission(awsProperties, requestBuilder);
+    S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder);
+    S3RequestUtil.configurePermission(s3FileIOProperties, requestBuilder);
 
     multipartUploadId = 
s3.createMultipartUpload(requestBuilder.build()).uploadId();
   }
@@ -312,7 +312,7 @@ class S3OutputStream extends PositionOutputStream {
                 
requestBuilder.contentMD5(BinaryUtils.toBase64(fileAndDigest.digest()));
               }
 
-              S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
+              S3RequestUtil.configureEncryption(s3FileIOProperties, 
requestBuilder);
 
               UploadPartRequest uploadRequest = requestBuilder.build();
 
@@ -429,8 +429,8 @@ class S3OutputStream extends PositionOutputStream {
         
requestBuilder.contentMD5(BinaryUtils.toBase64(completeMessageDigest.digest()));
       }
 
-      S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
-      S3RequestUtil.configurePermission(awsProperties, requestBuilder);
+      S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder);
+      S3RequestUtil.configurePermission(s3FileIOProperties, requestBuilder);
 
       s3.putObject(
           requestBuilder.build(),
diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java 
b/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java
index 1adbda66c0..62738e636b 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java
@@ -20,7 +20,6 @@ package org.apache.iceberg.aws.s3;
 
 import java.util.Locale;
 import java.util.function.Function;
-import org.apache.iceberg.aws.AwsProperties;
 import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
 import software.amazon.awssdk.services.s3.model.GetObjectRequest;
 import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
@@ -40,9 +39,9 @@ public class S3RequestUtil {
   private S3RequestUtil() {}
 
   static void configureEncryption(
-      AwsProperties awsProperties, PutObjectRequest.Builder requestBuilder) {
+      S3FileIOProperties s3FileIOProperties, PutObjectRequest.Builder 
requestBuilder) {
     configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         requestBuilder::serverSideEncryption,
         requestBuilder::ssekmsKeyId,
         requestBuilder::sseCustomerAlgorithm,
@@ -51,9 +50,9 @@ public class S3RequestUtil {
   }
 
   static void configureEncryption(
-      AwsProperties awsProperties, CreateMultipartUploadRequest.Builder 
requestBuilder) {
+      S3FileIOProperties s3FileIOProperties, 
CreateMultipartUploadRequest.Builder requestBuilder) {
     configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         requestBuilder::serverSideEncryption,
         requestBuilder::ssekmsKeyId,
         requestBuilder::sseCustomerAlgorithm,
@@ -62,9 +61,9 @@ public class S3RequestUtil {
   }
 
   static void configureEncryption(
-      AwsProperties awsProperties, UploadPartRequest.Builder requestBuilder) {
+      S3FileIOProperties s3FileIOProperties, UploadPartRequest.Builder 
requestBuilder) {
     configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         NULL_SSE_SETTER,
         NULL_STRING_SETTER,
         requestBuilder::sseCustomerAlgorithm,
@@ -73,9 +72,9 @@ public class S3RequestUtil {
   }
 
   static void configureEncryption(
-      AwsProperties awsProperties, GetObjectRequest.Builder requestBuilder) {
+      S3FileIOProperties s3FileIOProperties, GetObjectRequest.Builder 
requestBuilder) {
     configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         NULL_SSE_SETTER,
         NULL_STRING_SETTER,
         requestBuilder::sseCustomerAlgorithm,
@@ -84,9 +83,9 @@ public class S3RequestUtil {
   }
 
   static void configureEncryption(
-      AwsProperties awsProperties, HeadObjectRequest.Builder requestBuilder) {
+      S3FileIOProperties s3FileIOProperties, HeadObjectRequest.Builder 
requestBuilder) {
     configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         NULL_SSE_SETTER,
         NULL_STRING_SETTER,
         requestBuilder::sseCustomerAlgorithm,
@@ -96,52 +95,53 @@ public class S3RequestUtil {
 
   @SuppressWarnings("ReturnValueIgnored")
   static void configureEncryption(
-      AwsProperties awsProperties,
+      S3FileIOProperties s3FileIOProperties,
       Function<ServerSideEncryption, S3Request.Builder> encryptionSetter,
       Function<String, S3Request.Builder> kmsKeySetter,
       Function<String, S3Request.Builder> customAlgorithmSetter,
       Function<String, S3Request.Builder> customKeySetter,
       Function<String, S3Request.Builder> customMd5Setter) {
 
-    switch (awsProperties.s3FileIoSseType().toLowerCase(Locale.ENGLISH)) {
-      case AwsProperties.S3FILEIO_SSE_TYPE_NONE:
+    switch (s3FileIOProperties.sseType().toLowerCase(Locale.ENGLISH)) {
+      case S3FileIOProperties.SSE_TYPE_NONE:
         break;
 
-      case AwsProperties.S3FILEIO_SSE_TYPE_KMS:
+      case S3FileIOProperties.SSE_TYPE_KMS:
         encryptionSetter.apply(ServerSideEncryption.AWS_KMS);
-        kmsKeySetter.apply(awsProperties.s3FileIoSseKey());
+        kmsKeySetter.apply(s3FileIOProperties.sseKey());
         break;
 
-      case AwsProperties.S3FILEIO_SSE_TYPE_S3:
+      case S3FileIOProperties.SSE_TYPE_S3:
         encryptionSetter.apply(ServerSideEncryption.AES256);
         break;
 
-      case AwsProperties.S3FILEIO_SSE_TYPE_CUSTOM:
+      case S3FileIOProperties.SSE_TYPE_CUSTOM:
         // setters for SSE-C exist for all request builders, no need to check 
null
         customAlgorithmSetter.apply(ServerSideEncryption.AES256.name());
-        customKeySetter.apply(awsProperties.s3FileIoSseKey());
-        customMd5Setter.apply(awsProperties.s3FileIoSseMd5());
+        customKeySetter.apply(s3FileIOProperties.sseKey());
+        customMd5Setter.apply(s3FileIOProperties.sseMd5());
         break;
 
       default:
         throw new IllegalArgumentException(
-            "Cannot support given S3 encryption type: " + 
awsProperties.s3FileIoSseType());
+            "Cannot support given S3 encryption type: " + 
s3FileIOProperties.sseType());
     }
   }
 
   static void configurePermission(
-      AwsProperties awsProperties, PutObjectRequest.Builder requestBuilder) {
-    configurePermission(awsProperties, requestBuilder::acl);
+      S3FileIOProperties s3FileIOProperties, PutObjectRequest.Builder 
requestBuilder) {
+    configurePermission(s3FileIOProperties, requestBuilder::acl);
   }
 
   static void configurePermission(
-      AwsProperties awsProperties, CreateMultipartUploadRequest.Builder 
requestBuilder) {
-    configurePermission(awsProperties, requestBuilder::acl);
+      S3FileIOProperties s3FileIOProperties, 
CreateMultipartUploadRequest.Builder requestBuilder) {
+    configurePermission(s3FileIOProperties, requestBuilder::acl);
   }
 
   @SuppressWarnings("ReturnValueIgnored")
   static void configurePermission(
-      AwsProperties awsProperties, Function<ObjectCannedACL, 
S3Request.Builder> aclSetter) {
-    aclSetter.apply(awsProperties.s3FileIoAcl());
+      S3FileIOProperties s3FileIOProperties,
+      Function<ObjectCannedACL, S3Request.Builder> aclSetter) {
+    aclSetter.apply(s3FileIOProperties.acl());
   }
 }
diff --git 
a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java 
b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java
index 68f0f99658..8b4f46a23a 100644
--- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java
+++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Map;
 import org.apache.iceberg.TestHelpers;
 import org.apache.iceberg.aws.lakeformation.LakeFormationAwsClientFactory;
+import org.apache.iceberg.aws.s3.S3FileIOProperties;
 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.Maps;
@@ -66,14 +67,14 @@ public class TestAwsClientFactories {
   @Test
   public void testS3FileIoCredentialsVerification() {
     Map<String, String> properties = Maps.newHashMap();
-    properties.put(AwsProperties.S3FILEIO_ACCESS_KEY_ID, "key");
+    properties.put(S3FileIOProperties.ACCESS_KEY_ID, "key");
 
     Assertions.assertThatThrownBy(() -> AwsClientFactories.from(properties))
         .isInstanceOf(ValidationException.class)
         .hasMessage("S3 client access key ID and secret access key must be set 
at the same time");
 
-    properties.remove(AwsProperties.S3FILEIO_ACCESS_KEY_ID);
-    properties.put(AwsProperties.S3FILEIO_SECRET_ACCESS_KEY, "secret");
+    properties.remove(S3FileIOProperties.ACCESS_KEY_ID);
+    properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, "secret");
 
     Assertions.assertThatThrownBy(() -> AwsClientFactories.from(properties))
         .isInstanceOf(ValidationException.class)
diff --git 
a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java 
b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java
index 0444015b76..b927760f66 100644
--- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java
+++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java
@@ -45,7 +45,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.iceberg.aws.AwsProperties;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.assertj.core.api.Assertions;
 import org.junit.After;
@@ -87,12 +86,12 @@ public class TestS3OutputStream {
   private final Path tmpDir = Files.createTempDirectory("s3fileio-test-");
   private final String newTmpDirectory = "/tmp/newStagingDirectory";
 
-  private final AwsProperties properties =
-      new AwsProperties(
+  private final S3FileIOProperties properties =
+      new S3FileIOProperties(
           ImmutableMap.of(
-              AwsProperties.S3FILEIO_MULTIPART_SIZE,
+              S3FileIOProperties.MULTIPART_SIZE,
               Integer.toString(5 * 1024 * 1024),
-              AwsProperties.S3FILEIO_STAGING_DIRECTORY,
+              S3FileIOProperties.STAGING_DIRECTORY,
               tmpDir.toString(),
               "s3.write.tags.abc",
               "123",
@@ -105,7 +104,7 @@ public class TestS3OutputStream {
 
   @Before
   public void before() {
-    properties.setS3ChecksumEnabled(false);
+    properties.setChecksumEnabled(false);
     createBucket(BUCKET);
   }
 
@@ -169,9 +168,9 @@ public class TestS3OutputStream {
 
   @Test
   public void testStagingDirectoryCreation() throws IOException {
-    AwsProperties newStagingDirectoryAwsProperties =
-        new AwsProperties(
-            ImmutableMap.of(AwsProperties.S3FILEIO_STAGING_DIRECTORY, 
newTmpDirectory));
+    S3FileIOProperties newStagingDirectoryAwsProperties =
+        new S3FileIOProperties(
+            ImmutableMap.of(S3FileIOProperties.STAGING_DIRECTORY, 
newTmpDirectory));
     S3OutputStream stream =
         new S3OutputStream(s3, randomURI(), newStagingDirectoryAwsProperties, 
nullMetrics());
     stream.close();
@@ -179,7 +178,7 @@ public class TestS3OutputStream {
 
   @Test
   public void testWriteWithChecksumEnabled() {
-    properties.setS3ChecksumEnabled(true);
+    properties.setChecksumEnabled(true);
     writeTest();
   }
 
@@ -248,7 +247,7 @@ public class TestS3OutputStream {
 
   private void checkUploadPartRequestContent(
       byte[] data, ArgumentCaptor<UploadPartRequest> 
uploadPartRequestArgumentCaptor) {
-    if (properties.isS3ChecksumEnabled()) {
+    if (properties.isChecksumEnabled()) {
       List<UploadPartRequest> uploadPartRequests =
           uploadPartRequestArgumentCaptor.getAllValues().stream()
               .sorted(Comparator.comparingInt(UploadPartRequest::partNumber))
@@ -263,17 +262,17 @@ public class TestS3OutputStream {
 
   private void checkPutObjectRequestContent(
       byte[] data, ArgumentCaptor<PutObjectRequest> 
putObjectRequestArgumentCaptor) {
-    if (properties.isS3ChecksumEnabled()) {
+    if (properties.isChecksumEnabled()) {
       List<PutObjectRequest> putObjectRequests = 
putObjectRequestArgumentCaptor.getAllValues();
       assertEquals(getDigest(data, 0, data.length), 
putObjectRequests.get(0).contentMD5());
     }
   }
 
   private void checkTags(ArgumentCaptor<PutObjectRequest> 
putObjectRequestArgumentCaptor) {
-    if (properties.isS3ChecksumEnabled()) {
+    if (properties.isChecksumEnabled()) {
       List<PutObjectRequest> putObjectRequests = 
putObjectRequestArgumentCaptor.getAllValues();
       String tagging = putObjectRequests.get(0).tagging();
-      assertEquals(getTags(properties.s3WriteTags()), tagging);
+      assertEquals(getTags(properties.writeTags()), tagging);
     }
   }
 
diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java 
b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java
index 516342025b..816ca4799a 100644
--- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java
+++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iceberg.aws.s3;
 
-import org.apache.iceberg.aws.AwsProperties;
 import org.junit.Assert;
 import org.junit.Test;
 import software.amazon.awssdk.services.s3.model.S3Request;
@@ -34,12 +33,12 @@ public class TestS3RequestUtil {
 
   @Test
   public void testConfigureServerSideCustomEncryption() {
-    AwsProperties awsProperties = new AwsProperties();
-    awsProperties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_CUSTOM);
-    awsProperties.setS3FileIoSseKey("key");
-    awsProperties.setS3FileIoSseMd5("md5");
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties();
+    s3FileIOProperties.setSseType(S3FileIOProperties.SSE_TYPE_CUSTOM);
+    s3FileIOProperties.setSseKey("key");
+    s3FileIOProperties.setSseMd5("md5");
     S3RequestUtil.configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         this::setServerSideEncryption,
         this::setKmsKeyId,
         this::setCustomAlgorithm,
@@ -54,10 +53,10 @@ public class TestS3RequestUtil {
 
   @Test
   public void testConfigureServerSideS3Encryption() {
-    AwsProperties awsProperties = new AwsProperties();
-    awsProperties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_S3);
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties();
+    s3FileIOProperties.setSseType(S3FileIOProperties.SSE_TYPE_S3);
     S3RequestUtil.configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         this::setServerSideEncryption,
         this::setKmsKeyId,
         this::setCustomAlgorithm,
@@ -72,11 +71,11 @@ public class TestS3RequestUtil {
 
   @Test
   public void testConfigureServerSideKmsEncryption() {
-    AwsProperties awsProperties = new AwsProperties();
-    awsProperties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_KMS);
-    awsProperties.setS3FileIoSseKey("key");
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties();
+    s3FileIOProperties.setSseType(S3FileIOProperties.SSE_TYPE_KMS);
+    s3FileIOProperties.setSseKey("key");
     S3RequestUtil.configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         this::setServerSideEncryption,
         this::setKmsKeyId,
         this::setCustomAlgorithm,
@@ -91,11 +90,11 @@ public class TestS3RequestUtil {
 
   @Test
   public void testConfigureEncryptionSkipNullSetters() {
-    AwsProperties awsProperties = new AwsProperties();
-    awsProperties.setS3FileIoSseType(AwsProperties.S3FILEIO_SSE_TYPE_KMS);
-    awsProperties.setS3FileIoSseKey("key");
+    S3FileIOProperties s3FileIOProperties = new S3FileIOProperties();
+    s3FileIOProperties.setSseType(S3FileIOProperties.SSE_TYPE_KMS);
+    s3FileIOProperties.setSseKey("key");
     S3RequestUtil.configureEncryption(
-        awsProperties,
+        s3FileIOProperties,
         v -> null,
         v -> null,
         this::setCustomAlgorithm,

Reply via email to