singhpk234 commented on issue #5414:
URL: https://github.com/apache/iceberg/issues/5414#issuecomment-1204812269

   I see now, why this would not work basically in iceberg we have relocated 
guava, from `com.google.common.collect.ImmutableMap` to 
`org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap`
   
   the serializer for guava in https://github.com/magro/kryo-serializers, won't 
work now as they are trying to make obj of 
`com.google.common.collect.ImmutableMap` during deserialization ref 
[CP](https://github.com/magro/kryo-serializers/blob/master/src/main/java/de/javakaffee/kryoserializers/guava/ImmutableMapSerializer.java#L7)
   
   I was able to make the test case above pass via writing a custom serializer 
which on deserialization creates 
`org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap` in 
KryoHelper ... 
   
   ``` java
   import com.google.common.collect.Maps;
   import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
   
       Kryo kryo = new KryoSerializer(new SparkConf()).newKryo();
   
       kryo.register(java.util.HashMap.class);
       final ImmutableRelocatedMapSerializer serializer = new 
ImmutableRelocatedMapSerializer();
       kryo.register(obj.getClass(), serializer);
   
     public static class ImmutableRelocatedMapSerializer extends
         Serializer<ImmutableMap> {
   
       private static final boolean DOES_NOT_ACCEPT_NULL = true;
       private static final boolean IMMUTABLE = true;
   
       public ImmutableRelocatedMapSerializer() {
         super(DOES_NOT_ACCEPT_NULL, IMMUTABLE);
       }
   
       @Override
       public void write(Kryo kryo, Output output, ImmutableMap object) {
         kryo.writeObject(output, Maps.newHashMap(object));
       }
   
       @Override
       public ImmutableMap read(Kryo kryo, Input input, Class<ImmutableMap> 
type) {
         Map map = kryo.readObject(input, HashMap.class);
         return ImmutableMap.copyOf(map);
       }
     }
   ```
   
   so in this case you might wanna do something like this here, to achieve kryo 
serialization .. 
   


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