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-10809][state] Include keyed state that is not from head operat… #7048
Conversation
…ors in state assignment
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 @StefanRRichter. I have some questions to understand the change.
@@ -156,15 +165,22 @@ public void invoke(Tuple2<Integer, Integer> value, Context context) throws Excep | |||
} | |||
} | |||
|
|||
private static class FromPartitionFileSource extends RichParallelSourceFunction<Tuple2<Integer, Integer>> { | |||
private static class FromPartitionFileSource extends RichParallelSourceFunction<Tuple2<Integer, Integer>> | |||
implements CheckpointedFunction, CheckpointListener { |
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.
Needs proper indentation
} | ||
} catch (EOFException ignore) { | ||
while (!isRestored) { |
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.
This while loop is a bit confusing for me.
Isn't isRestored
set only once in initializeState
?
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.
Yes, and that is ok. The loop is only there to prevent that the source will terminate before a successful checkpoint. After the failover, when isRestore == true
, we allow that the source can terminate directly to finish the job.
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.
Shouldn't then the checkFail
call alone be enough? The isRestored
flag is already checked there.
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.
Yeah, I guess we can just check it once in the beginning as an if
.
sumState = context.getOperatorStateStore().getListState( | ||
new ListStateDescriptor<>("sumState", Integer.class)); | ||
|
||
if (context.isRestored()) { |
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.
Where do we check whether or not we actually have restore something?
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.
Through through the restore flag and a validating sink? If state was dropped, the validation would go wrong.
@StefanRRichter I only have one final comment about the while loop. |
Thanks for the review @tzulitai! Merging. |
…ors in state assignment This closes #7048. Signed-off-by: Stefan Richter <s.richter@data-artisans.com>
…ors in state assignment This closes #7048. Signed-off-by: Stefan Richter <s.richter@data-artisans.com>
…ors in state assignment This closes apache#7048. Signed-off-by: Stefan Richter <s.richter@data-artisans.com>
…ors in state assignment This closes apache#7048. Signed-off-by: Stefan Richter <s.richter@data-artisans.com>
…ors in state assignment This closes apache#7048. Signed-off-by: Stefan Richter <s.richter@data-artisans.com>
…ors in state assignment
What is the purpose of the change
This PR includes keyed state that was not from a head operator (head of operator chain) in the state assignment. This fixes problems with restoring keyed state for operators after
DataStreamUtils.reinterpretAsKeyedStream
.Brief change log
Remove a check if keyed state is from a head operator in the state assignment algorithm. This was an optimization from the times where Flink only allowed keyed state in the head operators (like what happens after every
keyBy
).Verifying this change
Extended
ReinterpretDataStreamAsKeyedStreamITCase
with a recovery cycle to test proper state restore of non-head operators.Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: (no)Documentation