Kontinuation opened a new issue, #5974:
URL: https://github.com/apache/iceberg/issues/5974
### Apache Iceberg version
main (development)
### Query engine
Spark
### Please describe the bug 🐞
Spark SQL query (such as `CACHE TABLE <table_name>`) fails when Kryo
serialization was enabled. The command to launch spark-sql is:
```
spark-sql \
--master "spark://X1-Carbon:7077" \
--jars $ICEBERG_SPARK_RUNTIME_JAR \
--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
--conf
spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions
\
--conf
spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \
--conf spark.sql.catalog.spark_catalog.type=hive \
--conf spark.sql.catalog.demo=org.apache.iceberg.spark.SparkCatalog \
--conf spark.sql.catalog.demo.type=hadoop \
--conf
spark.sql.catalog.demo.warehouse=/home/kontinuation/local/iceberg/warehouse
...
```
Then run `CACHE TABLE demo.db.northwind_order;` to reproduce this problem:
```
spark-sql> CACHE TABLE demo.db.northwind_order;
22/10/13 12:43:10 WARN TaskSetManager: Lost task 2.0 in stage 0.0 (TID 2)
(192.168.97.137 executor 0): java.io.IOException:
com.esotericsoftware.kryo.KryoException: java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1477)
at
org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:228)
at
org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:105)
at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
at
org.apache.iceberg.spark.source.SparkScan$ReadTask.table(SparkScan.java:298)
at
org.apache.iceberg.spark.source.SparkScan$BatchReader.<init>(SparkScan.java:245)
at
org.apache.iceberg.spark.source.SparkScan$ReaderFactory.createColumnarReader(SparkScan.java:210)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.advanceToNextIter(DataSourceRDD.scala:79)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
at
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown
Source)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
Source)
at
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
at
org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:223)
at
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:302)
at
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1508)
at
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1435)
at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1499)
at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1322)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:376)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:327)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:136)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: com.esotericsoftware.kryo.KryoException:
java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:144)
at
com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
at
org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:306)
at
org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:338)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at
org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:340)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:259)
at scala.Option.getOrElse(Option.scala:189)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:233)
at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:228)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1470)
... 49 more
Caused by: java.lang.UnsupportedOperationException
at
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.put(ImmutableMap.java:780)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:162)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39)
at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
... 61 more
22/10/13 12:43:11 ERROR TaskSetManager: Task 1 in stage 0.0 failed 4 times;
aborting job
22/10/13 12:43:11 WARN TaskSetManager: Lost task 11.3 in stage 0.0 (TID 37)
(192.168.97.137 executor 0): TaskKilled (Stage cancelled)
22/10/13 12:43:11 WARN TaskSetManager: Lost task 9.2 in stage 0.0 (TID 40)
(192.168.97.137 executor 0): TaskKilled (Stage cancelled)
22/10/13 12:43:11 WARN TaskSetManager: Lost task 15.3 in stage 0.0 (TID 38)
(192.168.97.137 executor 0): TaskKilled (Stage cancelled)
22/10/13 12:43:11 WARN TaskSetManager: Lost task 14.3 in stage 0.0 (TID 41)
(192.168.97.137 executor 0): TaskKilled (Stage cancelled)
22/10/13 12:43:11 WARN TaskSetManager: Lost task 6.2 in stage 0.0 (TID 39)
(192.168.97.137 executor 0): TaskKilled (Stage cancelled)
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in
stage 0.0 failed 4 times, most recent failure: Lost task 1.3 in stage 0.0 (TID
34) (192.168.97.137 executor 0): java.io.IOException:
com.esotericsoftware.kryo.KryoException: java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1477)
at
org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:228)
at
org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:105)
at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
at
org.apache.iceberg.spark.source.SparkScan$ReadTask.table(SparkScan.java:298)
at
org.apache.iceberg.spark.source.SparkScan$BatchReader.<init>(SparkScan.java:245)
at
org.apache.iceberg.spark.source.SparkScan$ReaderFactory.createColumnarReader(SparkScan.java:210)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.advanceToNextIter(DataSourceRDD.scala:79)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
at
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown
Source)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
Source)
at
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
at
org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:223)
at
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:302)
at
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1508)
at
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1435)
at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1499)
at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1322)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:376)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:327)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:136)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: com.esotericsoftware.kryo.KryoException:
java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:144)
at
com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
at
org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:306)
at
org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:338)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at
org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:340)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:259)
at scala.Option.getOrElse(Option.scala:189)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:233)
at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:228)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1470)
... 49 more
Caused by: java.lang.UnsupportedOperationException
at
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.put(ImmutableMap.java:780)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:162)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39)
at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
... 61 more
Driver stacktrace:
at
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2672)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2608)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2607)
at
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
at
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
at
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2607)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1182)
at
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1182)
at scala.Option.foreach(Option.scala:407)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1182)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2860)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2802)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2791)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
Caused by: java.io.IOException: com.esotericsoftware.kryo.KryoException:
java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1477)
at
org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:228)
at
org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:105)
at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
at
org.apache.iceberg.spark.source.SparkScan$ReadTask.table(SparkScan.java:298)
at
org.apache.iceberg.spark.source.SparkScan$BatchReader.<init>(SparkScan.java:245)
at
org.apache.iceberg.spark.source.SparkScan$ReaderFactory.createColumnarReader(SparkScan.java:210)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.advanceToNextIter(DataSourceRDD.scala:79)
at
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
at
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown
Source)
at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
Source)
at
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
at
org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer$$anon$1.hasNext(InMemoryRelation.scala:118)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:223)
at
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:302)
at
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1508)
at
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1435)
at
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1499)
at
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1322)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:376)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:327)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
at
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:136)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.lang.Thread.run(Thread.java:829)
Caused by: com.esotericsoftware.kryo.KryoException:
java.lang.UnsupportedOperationException
Serialization trace:
refs (org.apache.iceberg.spark.source.SerializableTableWithSize)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:144)
at
com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
at
org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:306)
at
org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:338)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at
org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:340)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:259)
at scala.Option.getOrElse(Option.scala:189)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:233)
at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
at
org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:228)
at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1470)
... 49 more
Caused by: java.lang.UnsupportedOperationException
at
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.put(ImmutableMap.java:780)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:162)
at
com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39)
at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
at
com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
... 61 more
spark-sql>
```
This problem can be reproduced on latest Iceberg main branch with Apache
Spark 3.3, 3.2 and 3.1.
This regression was introduced by changes in [PR
4428](https://github.com/apache/iceberg/pull/4428/files), where `Map<String,
SnapshotRef> refs` was added to `SerializableTable` and immutable maps may
sneak in, which cannot be properly de-serialized by kryo serializer.
There were multiple issues related with kryo serialization of guava
immutable data structures in the past, such as
[5414](https://github.com/apache/iceberg/issues/5414),
[446](https://github.com/apache/iceberg/issues/446) and
[3586](https://github.com/apache/iceberg/issues/3586). The solution was to
replace immutable data structures with standard java collections where kryo
serialization may kick in.
--
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]