Skip to content
Closed
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
67 changes: 67 additions & 0 deletions core/src/main/java/org/apache/iceberg/BaseFile.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
Expand Down Expand Up @@ -632,4 +633,70 @@ public String toString() {
.add("content_size_in_bytes", contentSizeInBytes == null ? "null" : contentSizeInBytes)
.toString();
}

@SuppressWarnings("checkstyle:CyclomaticComplexity")
@Override
public boolean equals(Object other) {
if (other == null || getClass() != other.getClass()) {
return false;
}
BaseFile<?> baseFile = (BaseFile<?>) other;
return partitionSpecId == baseFile.partitionSpecId
&& fileSizeInBytes == baseFile.fileSizeInBytes
&& Objects.equals(partitionType, baseFile.partitionType)
&& Objects.equals(fileOrdinal, baseFile.fileOrdinal)
&& Objects.equals(manifestLocation, baseFile.manifestLocation)
&& content == baseFile.content
&& Objects.equals(filePath, baseFile.filePath)
&& format == baseFile.format
&& Objects.equals(partitionData, baseFile.partitionData)
&& Objects.equals(recordCount, baseFile.recordCount)
&& Objects.equals(dataSequenceNumber, baseFile.dataSequenceNumber)
&& Objects.equals(fileSequenceNumber, baseFile.fileSequenceNumber)
&& Objects.equals(columnSizes, baseFile.columnSizes)
&& Objects.equals(valueCounts, baseFile.valueCounts)
&& Objects.equals(nullValueCounts, baseFile.nullValueCounts)
&& Objects.equals(nanValueCounts, baseFile.nanValueCounts)
&& Objects.equals(lowerBounds, baseFile.lowerBounds)
&& Objects.equals(upperBounds, baseFile.upperBounds)
&& Objects.deepEquals(splitOffsets, baseFile.splitOffsets)
&& Objects.deepEquals(equalityIds, baseFile.equalityIds)
&& Objects.deepEquals(keyMetadata, baseFile.keyMetadata)
&& Objects.equals(sortOrderId, baseFile.sortOrderId)
&& Objects.equals(firstRowId, baseFile.firstRowId)
&& Objects.equals(referencedDataFile, baseFile.referencedDataFile)
&& Objects.equals(contentOffset, baseFile.contentOffset)
&& Objects.equals(contentSizeInBytes, baseFile.contentSizeInBytes);
}

@Override
public int hashCode() {
return Objects.hash(
partitionType,
fileOrdinal,
manifestLocation,
partitionSpecId,
content,
filePath,
format,
partitionData,
recordCount,
fileSizeInBytes,
dataSequenceNumber,
fileSequenceNumber,
columnSizes,
valueCounts,
nullValueCounts,
nanValueCounts,
lowerBounds,
upperBounds,
Arrays.hashCode(splitOffsets),
Arrays.hashCode(equalityIds),
Arrays.hashCode(keyMetadata),
sortOrderId,
firstRowId,
referencedDataFile,
contentOffset,
contentSizeInBytes);
}
}
116 changes: 116 additions & 0 deletions core/src/main/java/org/apache/iceberg/ContentFileAvroEncoder.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* 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.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.lang.reflect.Array;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.inmemory.InMemoryInputFile;
import org.apache.iceberg.inmemory.InMemoryOutputFile;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
import org.apache.iceberg.types.Types;

/**
* A utility class to encode {@link ContentFile} implementations as Avro in a backwards compatible
* way. It uses the same Avro encoding mechanism as {@link ManifestWriter} and {@link
* ManifestReader}. *
*/
public class ContentFileAvroEncoder {
private ContentFileAvroEncoder() {}

public static <T> byte[] encode(ContentFile<T>[] files) throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
DataOutputStream view = new DataOutputStream(out);

Map<Types.StructType, List<ContentFile<T>>> filesByPartitionType = Maps.newLinkedHashMap();
for (ContentFile<T> dataFile : files) {
Types.StructType partitionType = ((PartitionData) dataFile.partition()).getPartitionType();
filesByPartitionType
.computeIfAbsent(partitionType, ignoredSpec -> Lists.newArrayList())
.add(dataFile);
}
// Number of unique partition types
view.writeInt(filesByPartitionType.size());

for (Map.Entry<Types.StructType, List<ContentFile<T>>> entry :
filesByPartitionType.entrySet()) {
Types.StructType partitionType = entry.getKey();
List<ContentFile<T>> dataFiles = entry.getValue();
Schema fileSchema = new Schema(DataFile.getType(partitionType).fields());

String partitionSchema = SchemaParser.toJson(partitionType.asSchema());
view.writeUTF(partitionSchema);

InMemoryOutputFile outputFile = new InMemoryOutputFile();
try (FileAppender<ContentFile<T>> fileAppender =
InternalData.write(FileFormat.AVRO, outputFile).schema(fileSchema).build()) {
fileAppender.addAll(dataFiles);
}

byte[] serialisedFiles = outputFile.toByteArray();
view.writeInt(serialisedFiles.length);
view.write(serialisedFiles);
}

return out.toByteArray();
}

public static DataFile[] decodeDataFiles(byte[] serialized) throws IOException {
return decode(serialized, GenericDataFile.class);
}

public static DeleteFile[] decodeDeleteFiles(byte[] serialized) throws IOException {
return decode(serialized, GenericDeleteFile.class);
}

private static <T extends StructLike> T[] decode(byte[] serialized, Class<T> fileClass)
throws IOException {
DataInputStream view = new DataInputStream(new ByteArrayInputStream(serialized));
List<T> files = Lists.newArrayList();

int uniqueSpecTypes = view.readInt();
for (int i = 0; i < uniqueSpecTypes; i++) {
Schema partitionSchema = SchemaParser.fromJson(view.readUTF());
Schema fileSchema = new Schema(DataFile.getType(partitionSchema.asStruct()).fields());

byte[] fileBuffer = new byte[view.readInt()];
ByteStreams.readFully(view, fileBuffer);

try (CloseableIterable<T> reader =
InternalData.read(FileFormat.AVRO, new InMemoryInputFile(fileBuffer))
.project(fileSchema)
.setRootType(fileClass)
.setCustomType(DataFile.PARTITION_ID, PartitionData.class)
.build()) {
reader.forEach(files::add);
}
}

return files.toArray((T[]) Array.newInstance(fileClass, files.size()));
}
}
34 changes: 34 additions & 0 deletions core/src/main/java/org/apache/iceberg/io/WriteResult.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,13 @@
package org.apache.iceberg.io;

import java.io.Serializable;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.util.CharSequenceSet;
Expand Down Expand Up @@ -134,4 +137,35 @@ public WriteResult build() {
return new WriteResult(dataFiles, deleteFiles, referencedDataFiles, rewrittenDeleteFiles);
}
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("dataFiles", dataFiles)
.add("deleteFiles", deleteFiles)
.add("referencedDataFiles", referencedDataFiles)
.add("rewrittenDeleteFiles", rewrittenDeleteFiles)
.toString();
}

@Override
public boolean equals(Object other) {
if (other == null || getClass() != other.getClass()) {
return false;
}
WriteResult that = (WriteResult) other;
return Objects.deepEquals(dataFiles, that.dataFiles)
&& Objects.deepEquals(deleteFiles, that.deleteFiles)
&& Objects.deepEquals(referencedDataFiles, that.referencedDataFiles)
&& Objects.deepEquals(rewrittenDeleteFiles, that.rewrittenDeleteFiles);
}

@Override
public int hashCode() {
return Objects.hash(
Arrays.hashCode(dataFiles),
Arrays.hashCode(deleteFiles),
Arrays.hashCode(referencedDataFiles),
Arrays.hashCode(rewrittenDeleteFiles));
}
Comment on lines +141 to +170
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we strictly need equals / hashCode for the implementation?

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm asking because this is in core and potentially affects other engines.

Copy link
Contributor

Choose a reason for hiding this comment

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

Same here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm asking because this is in core and potentially affects other engines.

You're right, it's not strictly necessary for the implementation. However, there have been a few instances in this codebase of decomposing and manually checking subsets of WriteResult or DataFile/DeleteFile due to a lack of these methods, so they simplify DynamicWriteResultAggregator tests, as well as DynamicWriteResult and WriteResult serialisation tests.

I don't see any harm in adding them. Why would other engines rely on no implementations of equals / hashCode? In this codebase, apart from Flink, it's only used in Spark in SparkPositionDeltaWrite as a simple data class.

Copy link
Contributor

@pvary pvary Sep 18, 2025

Choose a reason for hiding this comment

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

It is very easy to rely on equals/hashCode inadvertently. Consider HsshSet, HashMap, etc.

Equals is typically used many places, but before this change it defaults to instance equality which is very different

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's clear. Do you have any specific examples where you think this could be problematic? I can't think of any use case where someone would use BaseFile implementations as keys in a HashMap or put them in a HashSet without meaningful equals/hashCode implementations.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@pvary I am happy to extract the equals/hashCode into a separate PR and use manual decomposition of properties of WriteResult and Data/DeleteFiles in tests in this PR if you prefer. Although I still believe it should be totally fine to add these methods to simplify the code.

}
58 changes: 58 additions & 0 deletions core/src/test/java/org/apache/iceberg/io/TestWriteResult.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.io;

import static org.apache.iceberg.TestBase.SPEC;
import static org.assertj.core.api.Assertions.assertThat;

import java.io.IOException;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileMetadata;
import org.apache.iceberg.TestHelpers;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;

public class TestWriteResult {
static final DataFile FILE_A =
DataFiles.builder(SPEC)
.withPath("/path/to/data-a.parquet")
.withFileSizeInBytes(10)
.withPartitionPath("data_bucket=0") // easy way to set partition data for now
.withRecordCount(1)
.build();
static final DeleteFile FILE_A_DELETES =
FileMetadata.deleteFileBuilder(SPEC)
.ofPositionDeletes()
.withPath("/path/to/data-a-deletes.parquet")
.withFileSizeInBytes(10)
.withPartitionPath("data_bucket=0") // easy way to set partition data for now
.withRecordCount(1)
.build();
private static final WriteResult WRITE_RESULT =
WriteResult.builder().addDataFiles(FILE_A).addDeleteFiles(FILE_A_DELETES).build();

@ParameterizedTest
@MethodSource("org.apache.iceberg.TestHelpers#serializers")
public void serialization(TestHelpers.RoundTripSerializer<WriteResult> roundTripSerializer)
throws IOException, ClassNotFoundException {
assertThat(roundTripSerializer.apply(WRITE_RESULT)).isEqualTo(WRITE_RESULT);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.iceberg.flink.sink;

import java.util.Arrays;
import java.util.List;
import java.util.NavigableMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.flink.annotation.Internal;
Expand All @@ -43,11 +42,7 @@ public CommitSummary(NavigableMap<Long, WriteResult> pendingResults) {
pendingResults.values().forEach(this::addWriteResult);
}

public void addAll(NavigableMap<Long, List<WriteResult>> pendingResults) {
pendingResults.values().forEach(writeResults -> writeResults.forEach(this::addWriteResult));
}

private void addWriteResult(WriteResult writeResult) {
public void addWriteResult(WriteResult writeResult) {
dataFilesCount.addAndGet(writeResult.dataFiles().length);
Arrays.stream(writeResult.dataFiles())
.forEach(
Expand Down
Loading