-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Fix commit idempotence of DynamicIcebergSink #14092
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
aiborodin
wants to merge
1
commit into
apache:main
from
aiborodin:fix-commit-idempotence-of-dynamic-sink
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Some comments aren't visible on the classic Files Changed page.
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
116 changes: 116 additions & 0 deletions
116
core/src/main/java/org/apache/iceberg/ContentFileAvroEncoder.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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())); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
58 changes: 58 additions & 0 deletions
58
core/src/test/java/org/apache/iceberg/io/TestWriteResult.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same here
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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
WriteResultorDataFile/DeleteFiledue to a lack of these methods, so they simplifyDynamicWriteResultAggregatortests, as well asDynamicWriteResultandWriteResultserialisation 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
SparkPositionDeltaWriteas a simple data class.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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
BaseFileimplementations as keys in a HashMap or put them in a HashSet without meaningful equals/hashCode implementations.There was a problem hiding this comment.
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.