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
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 3
"modification": 2
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,15 @@

import com.google.auto.value.AutoValue;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.FileFormat;
Expand All @@ -41,8 +45,11 @@
* encode/decode it. This class is an identical version that can be used as a PCollection element
* type.
*
* <p>Use {@link #from(DataFile, PartitionKey)} to create a {@link SerializableDataFile} and {@link
* #createDataFile(PartitionSpec)} to reconstruct the original {@link DataFile}.
* <p>NOTE: If you add any new fields here, you need to also update the {@link #equals} and {@link
* #hashCode()} methods.
*
* <p>Use {@link #from(DataFile, String)} to create a {@link SerializableDataFile} and {@link
* #createDataFile(Map)} to reconstruct the original {@link DataFile}.
*/
@DefaultSchema(AutoValueSchema.class)
@AutoValue
Expand Down Expand Up @@ -199,4 +206,86 @@ DataFile createDataFile(Map<Integer, PartitionSpec> partitionSpecs) {
}
return output;
}

@Override
public final boolean equals(@Nullable Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SerializableDataFile that = (SerializableDataFile) o;
return getPath().equals(that.getPath())
&& getFileFormat().equals(that.getFileFormat())
&& getRecordCount() == that.getRecordCount()
&& getFileSizeInBytes() == that.getFileSizeInBytes()
&& getPartitionPath().equals(that.getPartitionPath())
&& getPartitionSpecId() == that.getPartitionSpecId()
&& Objects.equals(getKeyMetadata(), that.getKeyMetadata())
&& Objects.equals(getSplitOffsets(), that.getSplitOffsets())
&& Objects.equals(getColumnSizes(), that.getColumnSizes())
&& Objects.equals(getValueCounts(), that.getValueCounts())
&& Objects.equals(getNullValueCounts(), that.getNullValueCounts())
&& Objects.equals(getNanValueCounts(), that.getNanValueCounts())
&& mapEquals(getLowerBounds(), that.getLowerBounds())
&& mapEquals(getUpperBounds(), that.getUpperBounds());
}

private static boolean mapEquals(
@Nullable Map<Integer, byte[]> map1, @Nullable Map<Integer, byte[]> map2) {
if (map1 == null && map2 == null) {
return true;
} else if (map1 == null || map2 == null) {
return false;
}
Equivalence<byte[]> byteArrayEquivalence =
new Equivalence<byte[]>() {
@Override
protected boolean doEquivalent(byte[] a, byte[] b) {
return Arrays.equals(a, b);
}

@Override
protected int doHash(byte[] bytes) {
return Arrays.hashCode(bytes);
}
};

return Maps.difference(map1, map2, byteArrayEquivalence).areEqual();
}

@Override
public final int hashCode() {
int hashCode =
Objects.hash(
getPath(),
getFileFormat(),
getRecordCount(),
getFileSizeInBytes(),
getPartitionPath(),
getPartitionSpecId(),
getKeyMetadata(),
getSplitOffsets(),
getColumnSizes(),
getValueCounts(),
getNullValueCounts(),
getNanValueCounts());
hashCode = 31 * hashCode + computeMapByteHashCode(getLowerBounds());
hashCode = 31 * hashCode + computeMapByteHashCode(getUpperBounds());
return hashCode;
}

private static int computeMapByteHashCode(@Nullable Map<Integer, byte[]> map) {
if (map == null) {
return 0;
}
int hashCode = 0;
for (Map.Entry<Integer, byte[]> entry : map.entrySet()) {
int keyHash = entry.getKey().hashCode();
int valueHash = Arrays.hashCode(entry.getValue()); // content-based hash code
hashCode += keyHash ^ valueHash;
}
return hashCode;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* 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.beam.sdk.io.iceberg;

import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.junit.Test;

/**
* Test for {@link SerializableDataFile}. More tests can be found in {@link
* org.apache.beam.sdk.io.iceberg.RecordWriterManagerTest}.
*/
public class SerializableDataFileTest {
static final Set<String> FIELDS_SET =
ImmutableSet.<String>builder()
.add("path")
.add("fileFormat")
.add("recordCount")
.add("fileSizeInBytes")
.add("partitionPath")
.add("partitionSpecId")
.add("keyMetadata")
.add("splitOffsets")
.add("columnSizes")
.add("valueCounts")
.add("nullValueCounts")
.add("nanValueCounts")
.add("lowerBounds")
.add("upperBounds")
.build();

@Test
public void testFieldsInEqualsMethodInSyncWithGetterFields() {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@kennknowles added a check here with a helpful error message

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

We talked offline, and I think using EqualityTester could also be a good way to do this. Though if all the examples we pass to it have the default value for new fields, it will still miss errors...

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Right, EqualsTester just simplifies testing the equals() method for different objects, but it does not help us determine if all fields are accounted for.

List<String> getMethodNames =
Arrays.stream(SerializableDataFile.class.getDeclaredMethods())
.map(Method::getName)
.filter(methodName -> methodName.startsWith("get"))
.collect(Collectors.toList());

List<String> lowerCaseFields =
FIELDS_SET.stream().map(String::toLowerCase).collect(Collectors.toList());
List<String> extras = new ArrayList<>();
for (String field : getMethodNames) {
if (!lowerCaseFields.contains(field.substring(3).toLowerCase())) {
extras.add(field);
}
}
if (!extras.isEmpty()) {
throw new IllegalStateException(
"Detected new field(s) added to SerializableDataFile: "
+ extras
+ "\nPlease include the new field(s) in SerializableDataFile's equals() and hashCode() methods, then add them "
+ "to this test class's FIELDS_SET.");
}
}
}
Loading