stevenzwu commented on code in PR #7171:
URL: https://github.com/apache/iceberg/pull/7171#discussion_r1193053642
##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java:
##########
@@ -440,10 +442,12 @@ private byte[] writeToManifest(long checkpointId) throws
IOException {
return EMPTY_MANIFEST_DATA;
}
+ // Refresh table to get the latest specs map
+ table.refresh();
Review Comment:
is this step necessary?
this is definitely not needed if we go with the read-only table approach for
partition spec.
##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java:
##########
@@ -73,21 +78,43 @@ static ManifestOutputFileFactory createOutputFileFactory(
attemptNumber);
}
+ /**
+ * Write the {@link WriteResult} to temporary manifest files.
+ *
+ * @param result all those DataFiles/DeleteFiles in this WriteResult should
be written with same
+ * partition spec
+ */
static DeltaManifests writeCompletedFiles(
- WriteResult result, Supplier<OutputFile> outputFileSupplier,
PartitionSpec spec)
+ WriteResult result,
+ Supplier<OutputFile> outputFileSupplier,
+ Map<Integer, PartitionSpec> specsById)
throws IOException {
ManifestFile dataManifest = null;
ManifestFile deleteManifest = null;
+ PartitionSpec spec = null;
Review Comment:
> We use a SerializableTable instance to create IcebergStreamWriter. The
PartitionSpec of SerializableTable is fixed and will not change after the job
started. While the PartitionSpec for IcebergFilesCommitter is refreshed with
the table snapshot changing. This could fail the fink sink job when updating
the partition spec in another job. Because we use the wrong partition spec to
write those DataFiles/DeleteFiles to ManifestFile.
your original description analyzed the problem very well. I am wondering if
we should just pass the same read-only `SerializableTable` to
`IcebergFilesCommitter` so that it also use the same table spec as the
`IcebergStreamingWriter/RowDataTaskWriterFactory`.
@hililiwei @Reo-LEI what do you think?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]