Myasuka commented on a change in pull request #19051:
URL: https://github.com/apache/flink/pull/19051#discussion_r826677119
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/AbstractStateChangeLogger.java
##########
@@ -145,9 +145,17 @@ protected void log(
@Nullable ThrowingConsumer<DataOutputViewStreamWrapper,
IOException> dataWriter,
Ns ns)
throws IOException {
+ log(op, dataWriter, ns, keyContext.getCurrentKeyGroupIndex());
+ }
+
+ protected void log(
+ StateChangeOperation op,
+ @Nullable ThrowingConsumer<DataOutputViewStreamWrapper,
IOException> dataWriter,
+ Ns ns,
+ int keyGroup)
Review comment:
I think the root cause is that we don't have the correct `currentKey`
during pqState#poll, if we could modify `InternalTimerServiceImpl` like below,
to set the current key before calling poll:
~~~ java
private void onProcessingTime(long time) throws Exception {
// null out the timer in case the Triggerable calls
registerProcessingTimeTimer()
// inside the callback.
nextTimer = null;
InternalTimer<K, N> timer;
while ((timer = processingTimeTimersQueue.peek()) != null &&
timer.getTimestamp() <= time) {
keyContext.setCurrentKey(timer.getKey());
processingTimeTimersQueue.poll();
triggerTarget.onProcessingTime(timer);
}
if (timer != null && nextTimer == null) {
nextTimer =
processingTimeService.registerTimer(
timer.getTimestamp(), this::onProcessingTime);
}
}
~~~
and
~~~ java
public void advanceWatermark(long time) throws Exception {
currentWatermark = time;
InternalTimer<K, N> timer;
while ((timer = eventTimeTimersQueue.peek()) != null &&
timer.getTimestamp() <= time) {
keyContext.setCurrentKey(timer.getKey());
eventTimeTimersQueue.poll();
triggerTarget.onEventTime(timer);
}
~~~
The test could also pass. I think this change looks better with less change
and avoid to compute the key group again.
--
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]