You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/31 04:34:18 UTC

[GitHub] [iceberg] HeartSaVioR opened a new issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

HeartSaVioR opened a new issue #1279:
URL: https://github.com/apache/iceberg/issues/1279


   I've encountered the issue against Spark 3 with Iceberg master branch (aad99b5f7954678193ee6c3fa69f68392b8b81dd) + custom patches (to change Hadoop/Hive/Spark versions).
   
   Stack trace follows:
   
   ```
   20/07/31 04:00:59 ERROR util.Utils: Exception encountered
   com.esotericsoftware.kryo.KryoException: java.lang.UnsupportedOperationException
   Serialization trace:
   tasks (org.apache.iceberg.BaseCombinedScanTask)
           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.readObject(Kryo.java:731)
           at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:391)
           at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:302)
           at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
           at com.twitter.chill.WrappedArraySerializer.read(WrappedArraySerializer.scala:35)
           at com.twitter.chill.WrappedArraySerializer.read(WrappedArraySerializer.scala:23)
           at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
           at org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:296)
           at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$2(ParallelCollectionRDD.scala:80)
           at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$2$adapted(ParallelCollectionRDD.scala:80)
           at org.apache.spark.util.Utils$.deserializeViaNestedStream(Utils.scala:170)
           at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$1(ParallelCollectionRDD.scala:80)
           at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
           at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1343)
           at org.apache.spark.rdd.ParallelCollectionPartition.readObject(ParallelCollectionRDD.scala:70)
           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 java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1170)
           at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2178)
           at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2069)
           at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
           at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2287)
           at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2211)
           at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2069)
           at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
           at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431)
           at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76)
           at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:115)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:405)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   Caused by: java.lang.UnsupportedOperationException
           at org.apache.iceberg.relocated.com.google.common.collect.ImmutableCollection.add(ImmutableCollection.java:244)
           at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134)
           at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40)
           at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
           at com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
           ... 35 more
   ```
   
   As I mentioned, I'm using custom version of Iceberg where Hadoop/Hive/Spark versions are different with Iceberg's dependency, but the issue seems to be known one in Guava & Kyro, so I guess this is general one.


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666947446


   Yeah it's ideal to have some way to run E2E tests against "non-local mode" of Spark (as these issues are runtime issues and we wouldn't be able to find them unless we give a shot), but not an easy thing to do. If there's a way to force serializing and deserializing tasks even in local mode in Spark (for testing), that would be pretty much helpful.


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666914375


   Unlike RewriteDataFilesAction, RewriteManifestsAction and RemoveOrphanFilesAction worked.


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666952656


   OK, I confirmed my patch fixed the Spark case and I believe it will address Flink case as well. I didn't write test code for faster POC test. I'll write test code and submit a PR. Thanks all for inputs!


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

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] aokolnychyi commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666928396


   We have considered building a custom registrator but decided not to do that for exactly the same reason you describe: users will have to set it and Kryo will not work out of the box. Instead, we got rid of Guava in places that were exposed to Spark.
   
   I think `RewriteDataFilesAction` is relatively new so we haven't caught it.


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

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] aokolnychyi commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666931857


   We would probably want to think how to test this better too to avoid such things in the future.


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

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] HeartSaVioR edited a comment on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666947446






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

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] chenjunjiedada commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666948367


   I also encountered this when building Flink table source, the `ImmutableList.copy()` produces `ImmutableSingletonList` instance which cannot be serialized even when I register the ImmutableList serializer with `de.javakaffee.kryo-serializers`. So I have to change it to `Lists.newArrayLists(tasks)`.


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-670283314


   Resolved via https://github.com/apache/iceberg/commit/e682ca59129ab668f163924b030280c3787db8f6


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666926834


   Looks like widely used approach is leveraging https://github.com/magro/kryo-serializers , but Iceberg's Guava version is too far, and we even shade the class. My guess is that we need to implement our own KryoRegistrator, but this means we require end users to add the class to their Spark config.


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

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] HeartSaVioR commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666929812


   From the stack trace it feels me that we may need to fix BaseCombinedScanTask to not store ImmutableList "instance", though fixing core module due to the issue of Spark doesn't feel cool. I'll try out and see whether it fixes the issue.


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

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] HeartSaVioR edited a comment on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666947446


   Yeah it's ideal to have some way to run E2E tests against "non-local mode" of Spark (as these issues are runtime issues and we wouldn't be able to find them unless we give a shot), but not an easy thing to do.
   Instead, if there's a way to force serializing and deserializing tasks even in local mode in Spark (for testing), that would be pretty much helpful. (Not sure Spark already has one or not.)


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

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] HeartSaVioR closed issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
HeartSaVioR closed issue #1279:
URL: https://github.com/apache/iceberg/issues/1279


   


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

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] aokolnychyi commented on issue #1279: Spark: Kyro throws exception on deserializing BaseCombinedScanTask

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on issue #1279:
URL: https://github.com/apache/iceberg/issues/1279#issuecomment-666931179


   Here are a few examples of what we did:
   - https://github.com/apache/iceberg/pull/560
   - https://github.com/apache/iceberg/pull/775


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

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