|
21 | 21 |
|
22 | 22 | import com.google.auto.value.AutoValue; |
23 | 23 | import java.nio.ByteBuffer; |
| 24 | +import java.util.Arrays; |
24 | 25 | import java.util.HashMap; |
25 | 26 | import java.util.List; |
26 | 27 | import java.util.Map; |
| 28 | +import java.util.Objects; |
27 | 29 | import org.apache.beam.sdk.schemas.AutoValueSchema; |
28 | 30 | import org.apache.beam.sdk.schemas.annotations.DefaultSchema; |
| 31 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence; |
| 32 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; |
29 | 33 | import org.apache.iceberg.DataFile; |
30 | 34 | import org.apache.iceberg.DataFiles; |
31 | 35 | import org.apache.iceberg.FileFormat; |
|
41 | 45 | * encode/decode it. This class is an identical version that can be used as a PCollection element |
42 | 46 | * type. |
43 | 47 | * |
44 | | - * <p>Use {@link #from(DataFile, PartitionKey)} to create a {@link SerializableDataFile} and {@link |
45 | | - * #createDataFile(PartitionSpec)} to reconstruct the original {@link DataFile}. |
| 48 | + * <p>NOTE: If you add any new fields here, you need to also update the {@link #equals} and {@link |
| 49 | + * #hashCode()} methods. |
| 50 | + * |
| 51 | + * <p>Use {@link #from(DataFile, String)} to create a {@link SerializableDataFile} and {@link |
| 52 | + * #createDataFile(Map)} to reconstruct the original {@link DataFile}. |
46 | 53 | */ |
47 | 54 | @DefaultSchema(AutoValueSchema.class) |
48 | 55 | @AutoValue |
@@ -199,4 +206,86 @@ DataFile createDataFile(Map<Integer, PartitionSpec> partitionSpecs) { |
199 | 206 | } |
200 | 207 | return output; |
201 | 208 | } |
| 209 | + |
| 210 | + @Override |
| 211 | + public final boolean equals(@Nullable Object o) { |
| 212 | + if (this == o) { |
| 213 | + return true; |
| 214 | + } |
| 215 | + if (o == null || getClass() != o.getClass()) { |
| 216 | + return false; |
| 217 | + } |
| 218 | + SerializableDataFile that = (SerializableDataFile) o; |
| 219 | + return getPath().equals(that.getPath()) |
| 220 | + && getFileFormat().equals(that.getFileFormat()) |
| 221 | + && getRecordCount() == that.getRecordCount() |
| 222 | + && getFileSizeInBytes() == that.getFileSizeInBytes() |
| 223 | + && getPartitionPath().equals(that.getPartitionPath()) |
| 224 | + && getPartitionSpecId() == that.getPartitionSpecId() |
| 225 | + && Objects.equals(getKeyMetadata(), that.getKeyMetadata()) |
| 226 | + && Objects.equals(getSplitOffsets(), that.getSplitOffsets()) |
| 227 | + && Objects.equals(getColumnSizes(), that.getColumnSizes()) |
| 228 | + && Objects.equals(getValueCounts(), that.getValueCounts()) |
| 229 | + && Objects.equals(getNullValueCounts(), that.getNullValueCounts()) |
| 230 | + && Objects.equals(getNanValueCounts(), that.getNanValueCounts()) |
| 231 | + && mapEquals(getLowerBounds(), that.getLowerBounds()) |
| 232 | + && mapEquals(getUpperBounds(), that.getUpperBounds()); |
| 233 | + } |
| 234 | + |
| 235 | + private static boolean mapEquals( |
| 236 | + @Nullable Map<Integer, byte[]> map1, @Nullable Map<Integer, byte[]> map2) { |
| 237 | + if (map1 == null && map2 == null) { |
| 238 | + return true; |
| 239 | + } else if (map1 == null || map2 == null) { |
| 240 | + return false; |
| 241 | + } |
| 242 | + Equivalence<byte[]> byteArrayEquivalence = |
| 243 | + new Equivalence<byte[]>() { |
| 244 | + @Override |
| 245 | + protected boolean doEquivalent(byte[] a, byte[] b) { |
| 246 | + return Arrays.equals(a, b); |
| 247 | + } |
| 248 | + |
| 249 | + @Override |
| 250 | + protected int doHash(byte[] bytes) { |
| 251 | + return Arrays.hashCode(bytes); |
| 252 | + } |
| 253 | + }; |
| 254 | + |
| 255 | + return Maps.difference(map1, map2, byteArrayEquivalence).areEqual(); |
| 256 | + } |
| 257 | + |
| 258 | + @Override |
| 259 | + public final int hashCode() { |
| 260 | + int hashCode = |
| 261 | + Objects.hash( |
| 262 | + getPath(), |
| 263 | + getFileFormat(), |
| 264 | + getRecordCount(), |
| 265 | + getFileSizeInBytes(), |
| 266 | + getPartitionPath(), |
| 267 | + getPartitionSpecId(), |
| 268 | + getKeyMetadata(), |
| 269 | + getSplitOffsets(), |
| 270 | + getColumnSizes(), |
| 271 | + getValueCounts(), |
| 272 | + getNullValueCounts(), |
| 273 | + getNanValueCounts()); |
| 274 | + hashCode = 31 * hashCode + computeMapByteHashCode(getLowerBounds()); |
| 275 | + hashCode = 31 * hashCode + computeMapByteHashCode(getUpperBounds()); |
| 276 | + return hashCode; |
| 277 | + } |
| 278 | + |
| 279 | + private static int computeMapByteHashCode(@Nullable Map<Integer, byte[]> map) { |
| 280 | + if (map == null) { |
| 281 | + return 0; |
| 282 | + } |
| 283 | + int hashCode = 0; |
| 284 | + for (Map.Entry<Integer, byte[]> entry : map.entrySet()) { |
| 285 | + int keyHash = entry.getKey().hashCode(); |
| 286 | + int valueHash = Arrays.hashCode(entry.getValue()); // content-based hash code |
| 287 | + hashCode += keyHash ^ valueHash; |
| 288 | + } |
| 289 | + return hashCode; |
| 290 | + } |
202 | 291 | } |
0 commit comments