-
Notifications
You must be signed in to change notification settings - Fork 13k
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-35157][runtime] Sources with watermark alignment get stuck once some subtasks finish #24757
base: master
Are you sure you want to change the base?
Conversation
…e some subtasks finish
hi, @1996fanrui would you mind reviewing this for me when you have a moment? Thank you very much! |
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.
Hi @elon-X , I'm wondering if "NoMoreSplitsEvent" would be appropriate to check if a subtask is finished.
The Split can be discovered dynamically. IIRC, kafka source subtask won't be FINISHED even if it doesn't have split for Flink Streaming Job. (This subtask may be assigned split after adding new kafka partition)
I prefer Gyula proposed solution: The solution could be to send out a max watermark event once the sources finish or to exclude them from the source coordinator
.
We need to find a proper code place(source is definitely finished) to send MaxTimestamp.
I guess
Line 452 in 046872c
return DataInputStatus.END_OF_INPUT; |
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java
Show resolved
Hide resolved
@@ -0,0 +1,52 @@ | |||
package org.apache.flink.test.streaming.api.datastream; |
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.
Apache license is missed here.
import static org.junit.Assert.assertThat; | ||
|
||
/** This ITCase class tests the behavior of task execution with watermark alignment. */ | ||
public class SubTaskFinishedWithWatermarkAlignmentITCase { |
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.
nit:
How about renaming it to WatermarkAlignmentITCase
?
It means all ITCases related to WatermarkAlignment can be written here. Currently, we only have one test method : testTaskFinishedWithWatermarkAlignmentExecution
.
final List<Long> result = stream.executeAndCollect(101); | ||
|
||
// Assert that the collected result contains all numbers from 0 to 100 in any order | ||
assertThat(result, containsInAnyOrder(LongStream.rangeClosed(0, 100).boxed().toArray())); |
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.
All new test should use junit5 and assertj.
What is the purpose of the change
Sources with watermark alignment get stuck once some subtasks finish, this PR solves this problem.
Brief change log
while some subtasks have been finished, the SourceOperator send Long.MAX_VALUE to SourceCoordinator, and SourceCoordinator checks whether subtasks have been finished before sending the event.
Verifying this change
This change added tests and can be verified as follows:
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: noDocumentation