JonasJ-ap commented on code in PR #5939:
URL: https://github.com/apache/iceberg/pull/5939#discussion_r991864782
##########
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:
> Can you plz add the complete stacktrace you see in Spark Serialization
failure with java serializer
Here is the stacktrace that I copied from AWS Glue Job's Error Log:
```
2022-10-09 01:56:21,890 ERROR [main] glue.ProcessLauncher
(Logging.scala:logError(94)): Exception in User Class
java.io.NotSerializableException:
software.amazon.awssdk.services.sts.model.AssumeRoleRequest
Serialization stack:
- object not serializable (class:
software.amazon.awssdk.services.sts.model.AssumeRoleRequest, value:
AssumeRoleRequest(RoleArn=arn:aws:iam::481640105715:role/jonasjiang_gluejob2,
RoleSessionName=iceberg-aws-d9c0fac0-e938-4867-b8e3-f7f160f11a82,
DurationSeconds=3600, Tags=[]))
- field (class: org.apache.iceberg.aws.AssumeRoleAwsClientFactory,
name: assumeRoleRequest, type: class
software.amazon.awssdk.services.sts.model.AssumeRoleRequest)
- object (class org.apache.iceberg.aws.AssumeRoleAwsClientFactory,
org.apache.iceberg.aws.AssumeRoleAwsClientFactory@5a07088b)
- element of array (index: 0)
- array (class [Ljava.lang.Object;, size 1)
- field (class: java.lang.invoke.SerializedLambda, name: capturedArgs,
type: class [Ljava.lang.Object;)
- 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)
- field (class: org.apache.iceberg.SerializableTable, name: io, type:
interface org.apache.iceberg.io.FileIO)
- object (class
org.apache.iceberg.spark.source.SerializableTableWithSize,
org.apache.iceberg.spark.source.SerializableTableWithSize@28d07fa4)
at
org.apache.spark.serializer.SerializationDebugger$.improveException(SerializationDebugger.scala:41)
at
org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:47)
at
org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$blockifyObject$4(TorrentBroadcast.scala:319)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
at
org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:321)
at
org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:138)
at
org.apache.spark.broadcast.TorrentBroadcast.<init>(TorrentBroadcast.scala:91)
at
org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:35)
at
org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:77)
at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1545)
at
org.apache.spark.api.java.JavaSparkContext.broadcast(JavaSparkContext.scala:546)
at
org.apache.iceberg.spark.source.SparkWrite.createWriterFactory(SparkWrite.java:164)
at
org.apache.iceberg.spark.source.SparkWrite.access$700(SparkWrite.java:92)
at
org.apache.iceberg.spark.source.SparkWrite$BaseBatchWrite.createBatchWriterFactory(SparkWrite.java:241)
at
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:348)
at
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:336)
at
org.apache.spark.sql.execution.datasources.v2.AtomicReplaceTableAsSelectExec.writeWithV2(WriteToDataSourceV2Exec.scala:178)
at
org.apache.spark.sql.execution.datasources.v2.TableWriteExecHelper.$anonfun$writeToTable$1(WriteToDataSourceV2Exec.scala:476)
at
org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1473)
at
org.apache.spark.sql.execution.datasources.v2.TableWriteExecHelper.writeToTable(WriteToDataSourceV2Exec.scala:465)
at
org.apache.spark.sql.execution.datasources.v2.TableWriteExecHelper.writeToTable$(WriteToDataSourceV2Exec.scala:460)
at
org.apache.spark.sql.execution.datasources.v2.AtomicReplaceTableAsSelectExec.writeToTable(WriteToDataSourceV2Exec.scala:178)
at
org.apache.spark.sql.execution.datasources.v2.AtomicReplaceTableAsSelectExec.run(WriteToDataSourceV2Exec.scala:209)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:40)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:40)
at
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.doExecute(V2CommandExec.scala:55)
at
org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:185)
at
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:223)
at
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at
org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:220)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:181)
at
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:134)
at
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:133)
at
org.apache.spark.sql.DataFrameWriterV2.$anonfun$runCommand$1(DataFrameWriterV2.scala:196)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107)
at
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232)
at
org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:110)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:135)
at
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107)
at
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:135)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:253)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:134)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68)
at
org.apache.spark.sql.DataFrameWriterV2.runCommand(DataFrameWriterV2.scala:196)
at
org.apache.spark.sql.DataFrameWriterV2.internalReplace(DataFrameWriterV2.scala:213)
at
org.apache.spark.sql.DataFrameWriterV2.createOrReplace(DataFrameWriterV2.scala:133)
at GlueApp$.main(assumeroletest2.scala:30)
at GlueApp.main(assumeroletest2.scala)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
com.amazonaws.services.glue.SparkProcessLauncherPlugin.invoke(ProcessLauncher.scala:48)
at
com.amazonaws.services.glue.SparkProcessLauncherPlugin.invoke$(ProcessLauncher.scala:48)
at
com.amazonaws.services.glue.ProcessLauncher$$anon$1.invoke(ProcessLauncher.scala:78)
at
com.amazonaws.services.glue.ProcessLauncher.launch(ProcessLauncher.scala:143)
at
com.amazonaws.services.glue.ProcessLauncher$.main(ProcessLauncher.scala:30)
at
com.amazonaws.services.glue.ProcessLauncher.main(ProcessLauncher.scala)
```
--
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]