Skip to content

Commit 1f47712

Browse files
committed
simplify validation
1 parent 1c0f7d7 commit 1f47712

File tree

2 files changed

+38
-3
lines changed

2 files changed

+38
-3
lines changed

sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java

Lines changed: 37 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,9 @@
2222
import java.util.HashMap;
2323
import java.util.List;
2424
import java.util.Map;
25+
import java.util.Set;
2526
import java.util.UUID;
27+
import java.util.stream.Collectors;
2628
import org.apache.beam.sdk.coders.KvCoder;
2729
import org.apache.beam.sdk.coders.StringUtf8Coder;
2830
import org.apache.beam.sdk.metrics.Counter;
@@ -38,6 +40,8 @@
3840
import org.apache.beam.sdk.util.Preconditions;
3941
import org.apache.beam.sdk.values.KV;
4042
import org.apache.beam.sdk.values.PCollection;
43+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
44+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Streams;
4145
import org.apache.iceberg.AppendFiles;
4246
import org.apache.iceberg.DataFile;
4347
import org.apache.iceberg.FileFormat;
@@ -128,13 +132,12 @@ public void processElement(
128132
BoundedWindow window)
129133
throws IOException {
130134
String tableStringIdentifier = element.getKey();
135+
Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey()));
131136
Iterable<FileWriteResult> fileWriteResults = element.getValue();
132-
if (!fileWriteResults.iterator().hasNext()) {
137+
if (shouldSkip(table, fileWriteResults)) {
133138
return;
134139
}
135140

136-
Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey()));
137-
138141
// vast majority of the time, we will simply append data files.
139142
// in the rare case we get a batch that contains multiple partition specs, we will group
140143
// data into manifest files and append.
@@ -211,5 +214,36 @@ private ManifestWriter<DataFile> createManifestWriter(
211214
tableLocation, manifestFilePrefix, uuid, spec.specId()));
212215
return ManifestFiles.write(spec, io.newOutputFile(location));
213216
}
217+
218+
// If the process call fails immediately after a successful commit, it gets retried with
219+
// the same data, possibly leading to data duplication.
220+
// To mitigate, we skip the current batch of files if it matches the most recently committed
221+
// batch.
222+
//
223+
// TODO(ahmedabu98): This does not cover concurrent writes from other pipelines, where the
224+
// "last successful snapshot" might reflect commits from other sources. Ideally, we would make
225+
// this stateful, but that is update incompatible.
226+
// TODO(ahmedabu98): add load test pipelines with intentional periodic crashing
227+
private boolean shouldSkip(Table table, Iterable<FileWriteResult> fileWriteResults) {
228+
if (table.currentSnapshot() == null) {
229+
return false;
230+
}
231+
if (!fileWriteResults.iterator().hasNext()) {
232+
return true;
233+
}
234+
235+
Set<String> filesCommittedLastSnapshot =
236+
Streams.stream(table.currentSnapshot().addedDataFiles(table.io()))
237+
.map(DataFile::path)
238+
.map(CharSequence::toString)
239+
.collect(Collectors.toSet());
240+
241+
// Check if the current batch is identical to the most recently committed batch.
242+
// Upstream GBK means we always get the same batch of files on retry,
243+
// so a single overlapping file means the whole batch is identical.
244+
return Iterables.size(fileWriteResults) == filesCommittedLastSnapshot.size()
245+
&& filesCommittedLastSnapshot.contains(
246+
fileWriteResults.iterator().next().getSerializableDataFile().getPath());
247+
}
214248
}
215249
}

settings.gradle.kts

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -363,3 +363,4 @@ include("sdks:java:io:iceberg:bqms")
363363
findProject(":sdks:java:io:iceberg:bqms")?.name = "bqms"
364364
include("it:clickhouse")
365365
findProject(":it:clickhouse")?.name = "clickhouse"
366+
include("testing")

0 commit comments

Comments
 (0)