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 2021/02/20 07:31:43 UTC

[GitHub] [iceberg] zhangjun0x01 opened a new pull request #2258: Core : remove the UnmodifiableMap

zhangjun0x01 opened a new pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258


   Now, in `BaseFile`, there are some metrics, like `nullValueCounts`, `nanValueCounts` etc, they use `UnmodifiableMap`, resulting in flink serialization problems. For example, the `RewriteDataFilesAction` and streaming read function will failed,the issue is here(#2219 ). I checked some information and found that [spark has similar problems if using kryo.](https://stackoverflow.com/questions/46818293/how-to-set-unmodifiable-collection-serializer-of-kryo-in-spark-code),
   so I remove the `UnmodifiableMap`, and I tested that, the `RewriteDataFilesAction` will not throw exception.
   
   what do you think about this ? @rdblue 
   
   


----------------------------------------------------------------
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] yyanyy commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r595492248



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       I came across https://github.com/apache/iceberg/pull/775 which seems to be solving the same problem that could potentially be reused here. 




----------------------------------------------------------------
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] rdblue commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r588809379



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       @yyanyy is right. This is a method for defensive copies. I think it can be fine to do this, but we want to provide the same assurance that the data won't be modified. We can do that by moving the unmodifiable map creation to other places, but that is a big change and will result in a lot more object creation.
   
   Another option is to use an alternative wrapper class that can be cleanly serialized. Elsewhere in this class we use `SerializableByteBufferMap`. Does that work with Kryo? If so, can we use the same pattern here? Is that map modifiable?




----------------------------------------------------------------
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] openinx commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r595682774



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       The only way that I could think of is replacing the `Map<Integer, Long>` to an self-defined immutable map which won't implement the `java.util.Map` interface,  in this way we don't have to depend on the kryo's internal `MapSerializer` (It will call the `put` to add element into immutable collection).




----------------------------------------------------------------
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] zhangjun0x01 commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r583327547



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       Mainly, we use the `CombinedScanTask` class in many places, such as [RowDataIterator](https://github.com/apache/iceberg/blob/master/flink/src/main/java/org/apache/iceberg/flink/source/RowDataIterator.java#L57) for reading data. The bottom layer of the `CombinedScanTask ` class contains `BaseFile`. When we doing deserialize, because we cannot put the data into unmodifiableMap, it causes an exception ,  the deserialization failed, causing the program to fail. I tried other methods, but I did not find a suitable solution for the time being




----------------------------------------------------------------
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] zhangjun0x01 commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r589876894



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       I will find again if there are other serializers, or if we can  custom serializers




----------------------------------------------------------------
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] openinx commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r595671361



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       +1 to fix this BaseFile's kryo serialization issues.  I encountered the same issue when preparing the rewrite action for v2 ( https://github.com/apache/iceberg/pull/2303/files#diff-b51f408c19404436a1cf77f1f31b0a87e5ec1404295f576fc3ca90849de90da0R436).  I read the patch #775,  it just change the `com.google.common.collect.ImmutableList` to `java.util.Collections.UnmodifiableList`, both them does not support the `add` method, so how did the PR fix the kryo issue ?  I did not get the point.
   
   




----------------------------------------------------------------
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] yyanyy commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r583317366



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       I suspect that this was for defensive copy, are there any other unmodifiableMap/immutable map implementations that do not cause kryo serialization problem? Or is it possible to register extra kryo serializers in Flink so that we don't potentially run into similar problem 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] zhangjun0x01 commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r583328305



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       I remove the unmodifiableMap ,it is work fine,But I am not sure if it will affect other functions




----------------------------------------------------------------
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] openinx commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r595683276



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       Btw,  for this fix,  I think we have to provide an unit test to reproduce the kryo issue so that we don't have to fix it again and again in 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] rdblue commented on a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r588809379



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       @yyanyy is right. This is a method for defensive copies. I think it can be fine to do this, but we want to provide the same assurance that the data won't be modified. We can do that by moving the unmodifiable map creation to other places, but that is a big change and will result in a lot more object creation.
   
   Another option is to use an alternative wrapper class that can be cleanly serialized. Elsewhere in this class we use `SerializableByteBufferMap`. Does that work with Kryo? If so, can we use the same pattern here?
   
   That uses a proxy object. If that works, then the proxy object can be used to add items to the map and then the actual map could be unmodifiable.




----------------------------------------------------------------
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 a change in pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258#discussion_r604397586



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -434,7 +433,7 @@ public Integer sortOrderId() {
     if (map != null) {
       Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
       copy.putAll(map);
-      return Collections.unmodifiableMap(copy);

Review comment:
       I think Kryo in Spark somehow manages to handle unmodifiable collections. #775 came with a test and I also tested this recently in other places.




-- 
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] zhangjun0x01 closed pull request #2258: Core : remove the UnmodifiableMap

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 closed pull request #2258:
URL: https://github.com/apache/iceberg/pull/2258


   


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