-
Notifications
You must be signed in to change notification settings - Fork 13.1k
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-33398][runtime] Support switching from batch to stream mode for one input stream operator #23521
base: master
Are you sure you want to change the base?
Conversation
36a0166
to
edd6648
Compare
@flinkbot run azure |
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.
Thanks for the PR. Left some comments as below.
...a/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java
Outdated
Show resolved
Hide resolved
...a/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributesBuilder.java
Outdated
Show resolved
Hide resolved
.../src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java
Show resolved
Hide resolved
.../src/main/java/org/apache/flink/streaming/runtime/streamrecord/InternalRecordAttributes.java
Outdated
Show resolved
Hide resolved
flink-runtime/src/main/java/org/apache/flink/runtime/source/event/IsBacklogEvent.java
Outdated
Show resolved
Hide resolved
.../java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java
Show resolved
Hide resolved
...k-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestTriggerable.java
Outdated
Show resolved
Hide resolved
...ming-java/src/test/java/org/apache/flink/streaming/runtime/io/RecordAttributesValveTest.java
Outdated
Show resolved
Hide resolved
...src/test/java/org/apache/flink/streaming/api/operators/sort/SortingBacklogDataInputTest.java
Show resolved
Hide resolved
...src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java
Show resolved
Hide resolved
df7e9f5
to
756f929
Compare
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.
Thanks for the update. Left some comments as below.
Besides, this PR does not cover all the changes proposed in FLIP-327. According to offline discussions, we need more PRs to support state cache and built-in multi-input operators. Therefore, it might be better to create child tickets for FLINK-33202 that cover the whole implementation plan, and assign this PR to one of these child tickets.
...a/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
Show resolved
Hide resolved
...src/test/java/org/apache/flink/test/streaming/api/datastream/StreamingWithBacklogITCase.java
Show resolved
Hide resolved
.../java/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceImplTest.java
Outdated
Show resolved
Hide resolved
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LambdaTrigger.java
Outdated
Show resolved
Hide resolved
d83c464
to
a4013fc
Compare
@yunfengzhou-hub Thanks for the review! I updated the PR. Can you have another look? |
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.
Thanks for the update. Left some minor comments as below. @xintongsong Could you please take a look at this PR?
...time/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
Outdated
Show resolved
Hide resolved
...va/org/apache/flink/streaming/api/operators/InternalBacklogAwareTimerServiceManagerImpl.java
Show resolved
Hide resolved
...treaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java
Show resolved
Hide resolved
...treaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordAttributesValve.java
Outdated
Show resolved
Hide resolved
@xintongsong Could you help review this PR? |
Hi @Sxnan! Sorry, I understand that I'm not a reviewer, but it happened that I was testing functionality from this MR recently, and I found a bug (in my opinion). It concerns the logic of I suggest to have different counters for
WDYT? Also, it is good to have a test for aforementioned case. |
Hi @Smir. Thanks for trying this out! The RecordAttributesValve combines the RecodAttributes from different input channels from the same input. The input is considered in backlog state if and only if all the input channels are backlog = true. Otherwise, some non-backlog records will be considered as backlog records. Back to your case, where there is very little backlog data and it has parallelism greater than 1. It is possible that an input channel switches to non-backlog state before other input channels switch to backlog state. When that happens, we just process those data as if they are non-backlog data. |
Thanks for the response, @Sxnan. I got your points. BTW, the reason why delivery guarantees of Also, I should mention that there is another workaround that guarantees delivery of
The downside is that it is possible to have |
91c334d
to
ffd611b
Compare
flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java
Outdated
Show resolved
Hide resolved
flink-runtime/src/main/java/org/apache/flink/runtime/source/event/BacklogEvent.java
Outdated
Show resolved
Hide resolved
.../src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java
Outdated
Show resolved
Hide resolved
...ing-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java
Outdated
Show resolved
Hide resolved
...ing-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java
Outdated
Show resolved
Hide resolved
if (newKey == null) { | ||
currentWatermark = maxWatermarkDuringBacklog; | ||
} |
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.
What does newKey == null
mean? And why we need to update the current watermark under such condition?
public KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> getEventTimeTimersQueue() { | ||
return eventTimeTimersQueue; | ||
} |
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.
It feels hacky to expose an internal queue from one component and use it to construct another component. If the queue is meant to be shared, I think the correct way is to create the queue out side the two components and pass it in as constructor arguments.
* the timer service will be the maximum watermark during backlog processing. | ||
*/ | ||
@Internal | ||
public class InternalBacklogAwareTimerServiceImpl<K, N> implements InternalTimerService<N> { |
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.
TBH, I'm not a big fan of inherit. The problem is that, by directly accessing protected fields of the superclass from the subclasses, the contract between the classes becomes obscure. As subclass can see internals of the superclass, changes to the superclass can easily break things that subclasses depend on thus cause problems, and such issues are very hard to avoid.
In many cases, an inherit approach can be replaced by an equivalent approach based on composition, which has clearer contracts between classes. But admittedly it's no always possible and sometimes gets costly.
For the classes below, I guess I haven't spent enough time to explore whether it's possible to switch to a composition based approach. Do you think it's feasible?
BacklogTimeService
BatchExecutionInternalTimeService
InternalBacklogAwareTimerServiceManagerImpl
InternalTimeServiceManagerImpl
if (Duration.ZERO.equals( | ||
configuration.get( | ||
ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL_DURING_BACKLOG))) { |
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.
Same here for deciding whether to enable the mixed mode.
override def processRecordAttributes(recordAttributes: RecordAttributes): Unit = | ||
super.processRecordAttributes(recordAttributes) |
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.
I'm not familiar with Scala. What is this for? Shouldn't the method from the super class be called anyway if we don't override it?
…put and TwoInputStreamOperator
…ethod of PushingAsyncDataInput#DataOutput
…utes downstream when backlog status changed
…orting records during backlog processing
…ed during backlog processing
…t is disabled during backlog processing
After an offline discussion with @xintongsong, we will split this PR into two PRs. This PR will focus on optimizing the one input operator during backlog. The other PR will focus on propagating the RecordAttributes through the job graph. This PR should be merged after the other PR. |
What is the purpose of the change
This PR supports switching from batch to stream mode to improve the performance during processing backlog data for one input stream operator.
Brief change log
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)
: yesDocumentation