Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
39 changes: 17 additions & 22 deletions core/src/main/java/org/apache/iceberg/BaseFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
Expand All @@ -32,7 +31,6 @@
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ArrayUtil;
Expand Down Expand Up @@ -169,13 +167,12 @@ public PartitionData copy() {
this.recordCount = toCopy.recordCount;
this.fileSizeInBytes = toCopy.fileSizeInBytes;
if (fullCopy) {
// TODO: support lazy conversion to/from map
this.columnSizes = copy(toCopy.columnSizes);
this.valueCounts = copy(toCopy.valueCounts);
this.nullValueCounts = copy(toCopy.nullValueCounts);
this.nanValueCounts = copy(toCopy.nanValueCounts);
this.lowerBounds = SerializableByteBufferMap.wrap(copy(toCopy.lowerBounds));
this.upperBounds = SerializableByteBufferMap.wrap(copy(toCopy.upperBounds));
this.columnSizes = SerializableMap.copyOf(toCopy.columnSizes);
this.valueCounts = SerializableMap.copyOf(toCopy.valueCounts);
this.nullValueCounts = SerializableMap.copyOf(toCopy.nullValueCounts);
this.nanValueCounts = SerializableMap.copyOf(toCopy.nanValueCounts);
this.lowerBounds = SerializableByteBufferMap.wrap(SerializableMap.copyOf(toCopy.lowerBounds));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Later, we should check whether we need two wrappers, but it isn't a blocker here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do need SerializableByteBufferMap for now as byte buffers may not be serializable.

this.upperBounds = SerializableByteBufferMap.wrap(SerializableMap.copyOf(toCopy.upperBounds));
} else {
this.columnSizes = null;
this.valueCounts = null;
Expand Down Expand Up @@ -382,32 +379,32 @@ public long fileSizeInBytes() {

@Override
public Map<Integer, Long> columnSizes() {
return columnSizes;
return toReadableMap(columnSizes);
}

@Override
public Map<Integer, Long> valueCounts() {
return valueCounts;
return toReadableMap(valueCounts);
}

@Override
public Map<Integer, Long> nullValueCounts() {
return nullValueCounts;
return toReadableMap(nullValueCounts);
}

@Override
public Map<Integer, Long> nanValueCounts() {
return nanValueCounts;
return toReadableMap(nanValueCounts);
}

@Override
public Map<Integer, ByteBuffer> lowerBounds() {
return lowerBounds;
return toReadableMap(lowerBounds);
}

@Override
public Map<Integer, ByteBuffer> upperBounds() {
return upperBounds;
return toReadableMap(upperBounds);
}

@Override
Expand All @@ -430,13 +427,12 @@ public Integer sortOrderId() {
return sortOrderId;
}

private static <K, V> Map<K, V> copy(Map<K, V> map) {
if (map != null) {
Map<K, V> copy = Maps.newHashMapWithExpectedSize(map.size());
copy.putAll(map);
return Collections.unmodifiableMap(copy);
private static <K, V> Map<K, V> toReadableMap(Map<K, V> map) {
if (map instanceof SerializableMap) {
return ((SerializableMap<K, V>) map).immutableMap();
} else {
return map;
}
return null;
}

@Override
Expand All @@ -460,5 +456,4 @@ public String toString() {
.add("sort_order_id", sortOrderId)
.toString();
}

}
124 changes: 124 additions & 0 deletions core/src/main/java/org/apache/iceberg/SerializableMap.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.iceberg;

import java.io.Serializable;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;

public class SerializableMap<K, V> implements Map<K, V>, Serializable {

private final Map<K, V> copiedMap;
private Map<K, V> immutableMap;

SerializableMap() {
this.copiedMap = Maps.newHashMap();
}

private SerializableMap(Map<K, V> map) {
this.copiedMap = Maps.newHashMap();
this.copiedMap.putAll(map);
}

public static <K, V> SerializableMap<K, V> copyOf(Map<K, V> map) {
return map == null ? null : new SerializableMap<>(map);
}

public Map<K, V> immutableMap() {
if (immutableMap == null) {
immutableMap = Collections.unmodifiableMap(copiedMap);
}

return immutableMap;
}

@Override
public int size() {
return copiedMap.size();
}

@Override
public boolean isEmpty() {
return copiedMap.isEmpty();
}

@Override
public boolean containsKey(Object key) {
return copiedMap.containsKey(key);
}

@Override
public boolean containsValue(Object value) {
return copiedMap.containsValue(value);
}

@Override
public V get(Object key) {
return copiedMap.get(key);
}

@Override
public V put(K key, V value) {
return copiedMap.put(key, value);
}

@Override
public V remove(Object key) {
return copiedMap.remove(key);
}

@Override
public void putAll(Map<? extends K, ? extends V> m) {
copiedMap.putAll(m);
}

@Override
public void clear() {
copiedMap.clear();
}

@Override
public Set<K> keySet() {
return copiedMap.keySet();
}

@Override
public Collection<V> values() {
return copiedMap.values();
}

@Override
public Set<Entry<K, V>> entrySet() {
return copiedMap.entrySet();
}

@Override
public boolean equals(Object o) {
return copiedMap.equals(o);
}

@Override
public int hashCode() {
return copiedMap.hashCode();
}
}
Loading