You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2019/03/06 00:21:20 UTC

[incubator-iceberg] branch master updated: Deep copy maps and lists in GenericDataFile. (#106)

This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 1a78ed5  Deep copy maps and lists in GenericDataFile. (#106)
1a78ed5 is described below

commit 1a78ed5de193f0b7040e9d7da6c0da5ba96a2ac7
Author: Ryan Blue <rd...@users.noreply.github.com>
AuthorDate: Tue Mar 5 16:21:16 2019 -0800

    Deep copy maps and lists in GenericDataFile. (#106)
---
 .../java/com/netflix/iceberg/GenericDataFile.java  | 26 +++++++++++++++++-----
 1 file changed, 20 insertions(+), 6 deletions(-)

diff --git a/core/src/main/java/com/netflix/iceberg/GenericDataFile.java b/core/src/main/java/com/netflix/iceberg/GenericDataFile.java
index b5bfa06..d1000b6 100644
--- a/core/src/main/java/com/netflix/iceberg/GenericDataFile.java
+++ b/core/src/main/java/com/netflix/iceberg/GenericDataFile.java
@@ -20,6 +20,7 @@
 package com.netflix.iceberg;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.netflix.iceberg.avro.AvroSchemaUtil;
 import com.netflix.iceberg.types.Type;
@@ -188,13 +189,13 @@ class GenericDataFile
     this.fileSizeInBytes = toCopy.fileSizeInBytes;
     this.blockSizeInBytes = toCopy.blockSizeInBytes;
     this.fileOrdinal = toCopy.fileOrdinal;
-    this.sortColumns = toCopy.sortColumns;
+    this.sortColumns = copy(toCopy.sortColumns);
     // TODO: support lazy conversion to/from map
-    this.columnSizes = toCopy.columnSizes;
-    this.valueCounts = toCopy.valueCounts;
-    this.nullValueCounts = toCopy.nullValueCounts;
-    this.lowerBounds = toCopy.lowerBounds;
-    this.upperBounds = toCopy.upperBounds;
+    this.columnSizes = copy(toCopy.columnSizes);
+    this.valueCounts = copy(toCopy.valueCounts);
+    this.nullValueCounts = copy(toCopy.nullValueCounts);
+    this.lowerBounds = SerializableByteBufferMap.wrap(copy(toCopy.lowerBounds));
+    this.upperBounds = SerializableByteBufferMap.wrap(copy(toCopy.upperBounds));
     this.fromProjectionPos = toCopy.fromProjectionPos;
     this.keyMetadata = toCopy.keyMetadata == null ? null : ByteBuffers.copy(toCopy.keyMetadata);
   }
@@ -425,4 +426,17 @@ class GenericDataFile
         .toString();
   }
 
+  private static <K, V> Map<K, V> copy(Map<K, V> map) {
+    if (map != null) {
+      return ImmutableMap.copyOf(map);
+    }
+    return null;
+  }
+
+  private static <E> List<E> copy(List<E> list) {
+    if (list != null) {
+      return ImmutableList.copyOf(list);
+    }
+    return null;
+  }
 }