-
Notifications
You must be signed in to change notification settings - Fork 2.2k
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
Flink: Fix duplicate data with upsert writer in case of aborted checkpoints #10526
Conversation
@zhongqishang: Why would this solve the issue? In my mental model, this only changes when the files are closed. The files are added in the |
I am trying to solve this problem. This scenario is hard to reproduce. I want to re-express the background of the problem. The checkpoint was triggered twice in succession, and the previous checkpoint was canceled. Cancelling the checkpoint did not cause the flink job to fail. Therefore, the results of prepareSnapshotPreBarrier twice were generated in one commit. Moving to snapshot is just to ensure that flush success is bound to checkpoint success, so that the above scenario will not occur. Of course I'm not sure if this will solve the problem or if it will cause other problems. @pvary Thank you very much for your guidance. |
@zhongqishang: We have 2 data files with the same data. I suspect that the 1st data file is generated in the 1st checkpoint, and the 2nd data file is generated in the 2nd checkpoint. Could you confirm that the writers are restarted, or the source is restarted? Is the source is stateful, and the state of the source is handled correctly? Also, could you please check if the |
Just like you said.
Operator not restarted. The flink job is normal. |
Sadly, I don't understand your answer fully.
Do we have logs like these? |
I'm sorry for not expressing myself clearly. I confirmed that the 1st data file is generated in the 1st checkpoint, and the 2nd data file is generated in the 2nd checkpoint. I confirmed that the writers and the source are not restarted.
Not found the logs. I think the checkpoint is not executing |
Thanks for the clarification around the restarts.
That's strange. Could you please double check (log levels, TM logs)? These are the logs which should be there if the changes are committed by the Flink job. If the logs aren't there, you can cross-check by the snapshot summary. Here please check if the flink jobId, checkpointId, operatorId is there or missing on the commit. |
For #10431 (comment) log level is
This log does not exist in the all tm log. |
This means, that the Again.. are we sure that we don't get duplicated records from the input stream for whatever reasons? |
The task has upsert enabled.
As you described before. The problem occurs when the result of 2 As I described before, in the one commit, it will query 2 pieces of data with the same id. |
So in summary:
So the fix should be something along these lines:
@rodmeneses: Is this issue solved naturally by the new SinkV2 implemetation? |
0328342
to
f55d507
Compare
f55d507
to
0a90981
Compare
@pvary Can you give me some advice? |
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java
Outdated
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java
Show resolved
Hide resolved
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java
Show resolved
Hide resolved
b413f75
to
60d183d
Compare
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java
Show resolved
Hide resolved
@zhongqishang @pvary I have a uber question. let's say checkpoint N was cancelled or timed out and checkpoint N+1 completed successfully. In this case, do we know all the writer subtasks have flushed data files for checkpoint N and all write results have all been received by the committer? |
I think we only need to be sure, that every writer which has successfully closed on a given checkpoint is added in an Iceberg commit for the given checkpoint. What we should be aware, is that the Iceberg commit for the non-successful checkpoint might not contain everything which is received up to the checkpoint time. I think this is ok, since the users could check if the given checkpoint was successful or not |
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java
Show resolved
Hide resolved
@zhongqishang: Thanks for the patience. I revisited the PR again, and found one potential issue. Could you please check it? |
@zhongqishang: Could you please rebase? |
# Conflicts: # flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java # flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java
948a562
to
6b004f5
Compare
@pvary Already rebased, PTAL. |
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.
+1 from my side.
@stevenzwu: Any more comments?
Shall we add the documentation about the state compatibility/incompatibility in a different PR?
thanks @zhongqishang for the root cause and fix. and thanks @pvary for the review. @zhongqishang please create a back port PR to 1.18 and 1.20. also let's create a standing PR for the next 1.7 release notes ( |
@stevenzwu @pvary @zhongqishang As 1.6 is the last version that supports JDK8[1], and this patch will fix the data duplication problem which is critical, IMHO, we need to backport this to 1.6.x, what do you think about this? [1] https://lists.apache.org/thread/5qn6pyfmx6xp0w0cytldvk4kj71k410c |
@klion26: It's a tricky question, as the change includes Flink state changes. See: #10526 (comment) I would try to avoid such incompatibility in a patch release. |
@pvary thanks for the reply
Please let me what you think about this, thanks. |
* main: (208 commits) Docs: Fix Flink 1.20 support versions (apache#11065) Flink: Fix compile warning (apache#11072) Docs: Initial committer guidelines and requirements for merging (apache#10780) Core: Refactor ZOrderByteUtils (apache#10624) API: implement types timestamp_ns and timestamptz_ns (apache#9008) Build: Bump com.google.errorprone:error_prone_annotations (apache#11055) Build: Bump mkdocs-material from 9.5.33 to 9.5.34 (apache#11062) Flink: Backport PR apache#10526 to v1.18 and v1.20 (apache#11018) Kafka Connect: Disable publish tasks in runtime project (apache#11032) Flink: add unit tests for range distribution on bucket partition column (apache#11033) Spark 3.5: Use FileGenerationUtil in PlanningBenchmark (apache#11027) Core: Add benchmark for appending files (apache#11029) Build: Ignore benchmark output folders across all modules (apache#11030) Spec: Add RemovePartitionSpecsUpdate REST update type (apache#10846) Docs: bump latest version to 1.6.1 (apache#11036) OpenAPI, Build: Apply spotless to testFixtures source code (apache#11024) Core: Generate realistic bounds in benchmarks (apache#11022) Add REST Compatibility Kit (apache#10908) Flink: backport PR apache#10832 of inferring parallelism in FLIP-27 source (apache#11009) Docs: Add Druid docs url to sidebar (apache#10997) ...
resolve #10431