fengjiajie commented on PR #10695:
URL: https://github.com/apache/iceberg/pull/10695#issuecomment-2228065838
> @fengjiajie: This change was introduced, because in older Iceberg versions
the different statistics behaved differently. See: #7643
>
> The goal is that the `DataFile` should be immutable.
>
> I have 2 main issues with the proposed change:
>
> * It make the statistics mutable
> * It would introduce an extra copy of the statistics maps, which is
problematic from performance reasons.
>
> I think the main issue is that in your code the generated statistics maps
are not stored in `SerializableMap` and `SerializableByteBufferMap` objects.
This is the reason why the copy method falls back wrapping it to
`Collections.unmodifiableMap(map)` instead of calling `map.immutableMap()`. The
result of the former is not serializable with Kryo, while the result of the
later should work.
@pvary Thank you for providing the context behind this issue, I agree with
keeping DataFile immutable. However, I have a few points to add:
1. In the example above, I used `new HashMap` for simplicity. In reality,
the `Metrics` object is obtained through
`org.apache.iceberg.parquet.ParquetUtil`'s `fileMetrics` method, which also
uses `HashMap` internally.
2. Even if I modify `ParquetUtil.fileMetrics` to return a `SerializableMap`,
when calling `DataFiles.builder`'s `copy()` method, this `SerializableMap` is
still processed by `toReadableMap` (returning `((SerializableMap<K, V>)
map).immutableMap()`), and the result is still an `UnmodifiableMap`, which
fails during Kryo deserialization.
After considering this, it seems the core issue lies in Flink's Kryo being
unable to deserialize `UnmodifiableMap`. I can resolve this by registering a
custom serializer with Flink:
```java
Class<?> unmodifiableMapClass =
Class.forName("java.util.Collections$UnmodifiableMap");
env.getConfig().addDefaultKryoSerializer(unmodifiableMapClass,
UnmodifiableCollectionsSerializer.class);
```
I've tested this and it works. Does `ParquetUtil.fileMetrics` still need to
be modified to return `SerializableMap`? If not, I will close this pull
request.
--
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]