-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Commit both data files and delete files to iceberg transaction. #1939
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
Merged
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
14c028b
Flink: Commit both data files and delete files to iceberg transaction.
openinx b5d2b09
Minor changes.
openinx c007165
Addressing the comments.
openinx 1eae61a
Add unit tests to address the state compatibility issues.
openinx 0fdec67
Minor changes.
openinx 4e769b2
Add unit tests: addressing the case that commit two failure checkpoin…
openinx 0c6d008
Address the comments and add more unit tests.
openinx 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
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
71 changes: 71 additions & 0 deletions
71
flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.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,71 @@ | ||
| /* | ||
| * 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.flink.sink; | ||
|
|
||
| import java.util.List; | ||
| import org.apache.iceberg.ManifestFile; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
|
|
||
| class DeltaManifests { | ||
|
|
||
| private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; | ||
|
|
||
| private final ManifestFile dataManifest; | ||
| private final ManifestFile deleteManifest; | ||
| private final CharSequence[] referencedDataFiles; | ||
|
|
||
| DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { | ||
| this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); | ||
| } | ||
|
|
||
| DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { | ||
| Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); | ||
|
|
||
| this.dataManifest = dataManifest; | ||
| this.deleteManifest = deleteManifest; | ||
| this.referencedDataFiles = referencedDataFiles; | ||
| } | ||
|
|
||
| ManifestFile dataManifest() { | ||
| return dataManifest; | ||
| } | ||
|
|
||
| ManifestFile deleteManifest() { | ||
| return deleteManifest; | ||
| } | ||
|
|
||
| CharSequence[] referencedDataFiles() { | ||
| return referencedDataFiles; | ||
| } | ||
|
|
||
| List<ManifestFile> manifests() { | ||
| List<ManifestFile> manifests = Lists.newArrayListWithCapacity(2); | ||
| if (dataManifest != null) { | ||
| manifests.add(dataManifest); | ||
| } | ||
|
|
||
| if (deleteManifest != null) { | ||
| manifests.add(deleteManifest); | ||
| } | ||
|
|
||
| return manifests; | ||
| } | ||
| } |
122 changes: 122 additions & 0 deletions
122
flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.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,122 @@ | ||
| /* | ||
| * 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.flink.sink; | ||
|
|
||
| import java.io.ByteArrayInputStream; | ||
| import java.io.ByteArrayOutputStream; | ||
| import java.io.DataInputStream; | ||
| import java.io.DataOutputStream; | ||
| import java.io.IOException; | ||
| import org.apache.flink.core.io.SimpleVersionedSerializer; | ||
| import org.apache.iceberg.ManifestFile; | ||
| import org.apache.iceberg.ManifestFiles; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
||
| class DeltaManifestsSerializer implements SimpleVersionedSerializer<DeltaManifests> { | ||
| private static final int VERSION_1 = 1; | ||
| private static final int VERSION_2 = 2; | ||
| private static final byte[] EMPTY_BINARY = new byte[0]; | ||
|
|
||
| static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); | ||
|
|
||
| @Override | ||
| public int getVersion() { | ||
| return VERSION_2; | ||
| } | ||
|
|
||
| @Override | ||
| public byte[] serialize(DeltaManifests deltaManifests) throws IOException { | ||
| Preconditions.checkNotNull(deltaManifests, "DeltaManifests to be serialized should not be null"); | ||
|
|
||
| ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); | ||
| DataOutputStream out = new DataOutputStream(binaryOut); | ||
|
|
||
| byte[] dataManifestBinary = EMPTY_BINARY; | ||
| if (deltaManifests.dataManifest() != null) { | ||
| dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); | ||
| } | ||
|
|
||
| out.writeInt(dataManifestBinary.length); | ||
| out.write(dataManifestBinary); | ||
|
|
||
| byte[] deleteManifestBinary = EMPTY_BINARY; | ||
| if (deltaManifests.deleteManifest() != null) { | ||
| deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); | ||
| } | ||
|
|
||
| out.writeInt(deleteManifestBinary.length); | ||
| out.write(deleteManifestBinary); | ||
|
|
||
| CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); | ||
| out.writeInt(referencedDataFiles.length); | ||
| for (int i = 0; i < referencedDataFiles.length; i++) { | ||
| out.writeUTF(referencedDataFiles[i].toString()); | ||
| } | ||
|
|
||
| return binaryOut.toByteArray(); | ||
| } | ||
|
|
||
| @Override | ||
| public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { | ||
| if (version == VERSION_1) { | ||
| return deserializeV1(serialized); | ||
| } else if (version == VERSION_2) { | ||
| return deserializeV2(serialized); | ||
| } else { | ||
| throw new RuntimeException("Unknown serialize version: " + version); | ||
| } | ||
| } | ||
|
|
||
| private DeltaManifests deserializeV1(byte[] serialized) throws IOException { | ||
| return new DeltaManifests(ManifestFiles.decode(serialized), null); | ||
| } | ||
|
|
||
| private DeltaManifests deserializeV2(byte[] serialized) throws IOException { | ||
| ManifestFile dataManifest = null; | ||
| ManifestFile deleteManifest = null; | ||
|
|
||
| ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); | ||
| DataInputStream in = new DataInputStream(binaryIn); | ||
|
|
||
| int dataManifestSize = in.readInt(); | ||
| if (dataManifestSize > 0) { | ||
| byte[] dataManifestBinary = new byte[dataManifestSize]; | ||
| Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); | ||
|
|
||
| dataManifest = ManifestFiles.decode(dataManifestBinary); | ||
| } | ||
|
|
||
| int deleteManifestSize = in.readInt(); | ||
| if (deleteManifestSize > 0) { | ||
| byte[] deleteManifestBinary = new byte[deleteManifestSize]; | ||
| Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); | ||
|
|
||
| deleteManifest = ManifestFiles.decode(deleteManifestBinary); | ||
| } | ||
|
|
||
| int referenceDataFileNum = in.readInt(); | ||
| CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; | ||
| for (int i = 0; i < referenceDataFileNum; i++) { | ||
| referencedDataFiles[i] = in.readUTF(); | ||
| } | ||
|
|
||
| return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); | ||
| } | ||
| } |
48 changes: 0 additions & 48 deletions
48
flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestSerializer.java
This file was deleted.
Oops, something went wrong.
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
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.
just for my own education,
referencedDataFilesfromWriteResultdoesn't seem to be used (except for unit test). What is it for? do we need to serialize it too?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.
We should serialize it and add it to the commit. This is the set of files that is referenced by any positional delete, which identifies deleted rows by file and row position. The commit will validate that all of the files still exist in the table.
This isn't strictly needed for this use case because we know that the position deletes only refer to files that are created in this commit. Since the files are being added in the commit, it isn't possible for some other process to delete some of them from metadata. But it is still good to configure the commit properly in case this gets reused 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.
Thanks for the explanation, @rdblue . I think it's correct to validate the data files in
RowDelta#commit. Will provide an extra unit test to address it.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.
This unit test addressed the data files validation issue