openinx commented on a change in pull request #1939:
URL: https://github.com/apache/iceberg/pull/1939#discussion_r545700153
##########
File path:
flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java
##########
@@ -63,4 +66,53 @@ static ManifestOutputFileFactory
createOutputFileFactory(Table table, String fli
TableOperations ops = ((HasTableOperations) table).operations();
return new ManifestOutputFileFactory(ops, table.io(), table.properties(),
flinkJobId, subTaskId, attemptNumber);
}
+
+ static DeltaManifests writeCompletedFiles(WriteResult result,
+ Supplier<OutputFile>
outputFileSupplier,
+ PartitionSpec spec) throws
IOException {
+
+ ManifestFile dataManifest = null;
+ ManifestFile deleteManifest = null;
+
+ // Write the completed data files into a newly created data manifest file.
+ if (result.dataFiles() != null && result.dataFiles().length > 0) {
+ dataManifest = writeDataFiles(outputFileSupplier.get(), spec,
Lists.newArrayList(result.dataFiles()));
+ }
+
+ // Write the completed delete files into a newly created delete manifest
file.
+ if (result.deleteFiles() != null && result.deleteFiles().length > 0) {
+ OutputFile deleteManifestFile = outputFileSupplier.get();
+
+ ManifestWriter<DeleteFile> deleteManifestWriter =
ManifestFiles.writeDeleteManifest(FORMAT_V2, spec,
+ deleteManifestFile, DUMMY_SNAPSHOT_ID);
+ try (ManifestWriter<DeleteFile> writer = deleteManifestWriter) {
+ for (DeleteFile deleteFile : result.deleteFiles()) {
+ writer.add(deleteFile);
+ }
+ }
+
+ deleteManifest = deleteManifestWriter.toManifestFile();
+ }
+
+ return new DeltaManifests(dataManifest, deleteManifest);
Review comment:
We have a similar discussion
[here](https://github.com/apache/iceberg/pull/1185#discussion_r474420170).
Even if the `WriteResult` is empty ( NOT null, null means there's nobody
emitted a result to the `IcebergFilesCommitter`, while empty `WriteResult`
means the `IcebergStreamWriter` did not write any new data but still emit a
WriterResult with zero data files and zero delete files to downstream
`IcebergFilesCommitter`), we'd better to commit to iceberg txn so that the
flink streaming job won't be failure easily when expiring a old snapshot (since
that time we did not even write any new records).
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]