singhpk234 commented on code in PR #5939:
URL: https://github.com/apache/iceberg/pull/5939#discussion_r992486634


##########
aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java:
##########
@@ -71,6 +73,53 @@ public void testS3FileIoCredentialsVerification() {
         () -> AwsClientFactories.from(properties));
   }
 
+  @Test
+  public void testDefaultAwsClientFactorySerializable() {
+    Map<String, String> properties = Maps.newHashMap();
+    AwsClientFactory defaultAwsClientFactory = 
AwsClientFactories.from(properties);
+    byte[] serializedFactoryBytes = 
SerializationUtil.serializeToBytes(defaultAwsClientFactory);
+    AwsClientFactory deserializedClientFactory =
+        SerializationUtil.deserializeFromBytes(serializedFactoryBytes);
+    Assert.assertTrue(
+        "DefaultAwsClientFactory should be serializable",
+        deserializedClientFactory instanceof 
AwsClientFactories.DefaultAwsClientFactory);
+  }
+
+  @Test
+  public void testAssumeRoleAwsClientFactorySerializable() {
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(AwsProperties.CLIENT_FACTORY, 
AssumeRoleAwsClientFactory.class.getName());
+    properties.put(AwsProperties.CLIENT_ASSUME_ROLE_ARN, "arn::test");
+    properties.put(AwsProperties.CLIENT_ASSUME_ROLE_REGION, "us-east-1");
+    AwsClientFactory assumeRoleAwsClientFactory = 
AwsClientFactories.from(properties);
+    byte[] serializedFactoryBytes = 
SerializationUtil.serializeToBytes(assumeRoleAwsClientFactory);
+    AwsClientFactory deserializedClientFactory =
+        SerializationUtil.deserializeFromBytes(serializedFactoryBytes);
+    Assert.assertTrue(
+        "AssumeRoleAwsClientFactory should be serializable",
+        deserializedClientFactory instanceof AssumeRoleAwsClientFactory);
+  }
+
+  @Test
+  public void testLakeFormationAwsClientFactorySerializable() {

Review Comment:
   Thanks for sharing the stack trace
   
   > - object (class java.lang.invoke.SerializedLambda, 
SerializedLambda[capturingClass=class org.apache.iceberg.aws.s3.S3FileIO, 
functionalInterfaceMethod=org/apache/iceberg/util/SerializableSupplier.get:()Ljava/lang/Object;,
 implementation=invokeInterface 
org/apache/iceberg/aws/AwsClientFactory.s3:()Lsoftware/amazon/awssdk/services/s3/S3Client;,
 instantiatedMethodType=()Lsoftware/amazon/awssdk/services/s3/S3Client;, 
numCaptured=1])
        - writeReplace data (class: java.lang.invoke.SerializedLambda)
        - object (class 
org.apache.iceberg.aws.s3.S3FileIO$$Lambda$1017/1010480754, 
org.apache.iceberg.aws.s3.S3FileIO$$Lambda$1017/1010480754@11885841)
        - field (class: org.apache.iceberg.aws.s3.S3FileIO, name: s3, type: 
interface org.apache.iceberg.util.SerializableSupplier)
        - object (class org.apache.iceberg.aws.s3.S3FileIO, 
org.apache.iceberg.aws.s3.S3FileIO@64236be)
   
   As per above, S3FileIO becomes non-serializable as well, due to lambda 
holding reference to client-factory, we have UT's for S3FileIO serde with 
Default AWS factory but not with other client factories, but since we are Unit 
testing other client factories serde, we should be good here as well. 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to