You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Roman Khachatryan (Jira)" <ji...@apache.org> on 2022/03/10 20:12:00 UTC

[jira] [Comment Edited] (FLINK-26063) [Changelog] Incorrect key group logged for PQ.poll and remove

    [ https://issues.apache.org/jira/browse/FLINK-26063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17504555#comment-17504555 ] 

Roman Khachatryan edited comment on FLINK-26063 at 3/10/22, 8:11 PM:
---------------------------------------------------------------------

{quote}
The impact is relatively small for remove(): in the worst case, the operation will be ignored.
{quote}
The operation is actually *not* ignored and fails the re-scaling (both heap and rocksdb queue impl.):
{code}
java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:268)
	at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:106)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:700)
	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:676)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:643)
	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:917)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore keyed state backend for WindowOperator_cdf5528fc65ae6b8b6b126cfdfcc40dd_(1/4) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:160)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:346)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:164)
	... 11 more
Caused by: java.lang.IllegalArgumentException: key group from 0 to 32 does not contain 46
	at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:160)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.globalKeyGroupToLocalIndex(KeyGroupPartitionedPriorityQueue.java:191)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.computeKeyGroupIndex(KeyGroupPartitionedPriorityQueue.java:186)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.getKeyGroupSubHeapForElement(KeyGroupPartitionedPriorityQueue.java:179)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.remove(KeyGroupPartitionedPriorityQueue.java:129)
	at org.apache.flink.state.changelog.restore.PriorityQueueStateChangeApplier.apply(PriorityQueueStateChangeApplier.java:45)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.applyDataChange(ChangelogBackendLogApplier.java:222)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.applyOperation(ChangelogBackendLogApplier.java:96)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.apply(ChangelogBackendLogApplier.java:73)
	at org.apache.flink.state.changelog.restore.ChangelogBackendRestoreOperation.readBackendHandle(ChangelogBackendRestoreOperation.java:93)
	at org.apache.flink.state.changelog.restore.ChangelogBackendRestoreOperation.restore(ChangelogBackendRestoreOperation.java:74)
	at org.apache.flink.state.changelog.ChangelogStateBackend.restore(ChangelogStateBackend.java:225)
	at org.apache.flink.state.changelog.ChangelogStateBackend.createKeyedStateBackend(ChangelogStateBackend.java:148)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:329)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:168)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	... 13 more
{code}

Raising priority to Major and fixVersion to 1.15 (not Blocker to not block the release).


was (Author: roman_khachatryan):
{quote}
The impact is relatively small for remove(): in the worst case, the operation will be ignored.
{quote}
The operation is actually not ignored and fails the recovery (both heap and rocksdb queue impl.):
{code}
java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:255)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:268)
	at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:106)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:700)
	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:676)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:643)
	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:917)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore keyed state backend for WindowOperator_cdf5528fc65ae6b8b6b126cfdfcc40dd_(1/4) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:160)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:346)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:164)
	... 11 more
Caused by: java.lang.IllegalArgumentException: key group from 0 to 32 does not contain 46
	at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:160)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.globalKeyGroupToLocalIndex(KeyGroupPartitionedPriorityQueue.java:191)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.computeKeyGroupIndex(KeyGroupPartitionedPriorityQueue.java:186)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.getKeyGroupSubHeapForElement(KeyGroupPartitionedPriorityQueue.java:179)
	at org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue.remove(KeyGroupPartitionedPriorityQueue.java:129)
	at org.apache.flink.state.changelog.restore.PriorityQueueStateChangeApplier.apply(PriorityQueueStateChangeApplier.java:45)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.applyDataChange(ChangelogBackendLogApplier.java:222)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.applyOperation(ChangelogBackendLogApplier.java:96)
	at org.apache.flink.state.changelog.restore.ChangelogBackendLogApplier.apply(ChangelogBackendLogApplier.java:73)
	at org.apache.flink.state.changelog.restore.ChangelogBackendRestoreOperation.readBackendHandle(ChangelogBackendRestoreOperation.java:93)
	at org.apache.flink.state.changelog.restore.ChangelogBackendRestoreOperation.restore(ChangelogBackendRestoreOperation.java:74)
	at org.apache.flink.state.changelog.ChangelogStateBackend.restore(ChangelogStateBackend.java:225)
	at org.apache.flink.state.changelog.ChangelogStateBackend.createKeyedStateBackend(ChangelogStateBackend.java:148)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:329)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:168)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
	... 13 more
{code}

Raising priority to Major and fixVersion to 1.15 (not Blocker to not block the release).

> [Changelog] Incorrect key group logged for PQ.poll and remove
> -------------------------------------------------------------
>
>                 Key: FLINK-26063
>                 URL: https://issues.apache.org/jira/browse/FLINK-26063
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / State Backends
>    Affects Versions: 1.15.0
>            Reporter: Roman Khachatryan
>            Priority: Major
>             Fix For: 1.15.0
>
>
> Key group is logged so that state changes can be re-distributed or shuffled.
> It is currently obtained from keyContext during poll() and remove() operations.
> However, keyContext is not updated when dequeing processing time timers.
> The impact is relatively small for remove(): in the worst case, the operation will be ignored.
> poll() should probably be replaced with remove() anyways - see FLINK-26062.
> One way to solve this problem is to extract key group from the polled element - if it is a timer.
> cc: [~masteryhx], [~ym], [~yunta]



--
This message was sent by Atlassian Jira
(v8.20.1#820001)