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

Piotr Nowojski commented on FLINK-31963:
----------------------------------------

To clarify impact of this bug. This is a rare issue that can happen in every 
not backpressured job. The problem is that if the input buffers of a downstream 
subtask are empty AND the output buffers of the upstream subtask are not empty, 
then in-flight data are incorrectly restored from such checkpoint during a 
recovery attempt combined with rescaling. This can lead to variety of issues:
* ArrayIndexOutOfBoundException when downscaling (as reported here)
* in-flight records sent to incorrect downstream subtasks during scaling up or 
down. This for keyed exchanges will cause an immediate failure when trying to 
match key group on the downstream subtask. For non keyed exchanges the 
misalignment can remain undetected, causing incorrect results. 

Checkpoint itself is not corrupted, so recovery attempt without rescaling would 
work without without problems. Also recovery and rescaling from such checkpoint 
using a Flink version that has this bug fixed will also work correctly.

> java.lang.ArrayIndexOutOfBoundsException when scaling down with unaligned 
> checkpoints
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-31963
>                 URL: https://issues.apache.org/jira/browse/FLINK-31963
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.17.0, 1.16.1, 1.15.4, 1.18.0
>         Environment: Flink: 1.17.0
> FKO: 1.4.0
> StateBackend: RocksDB(Genetic Incremental Checkpoint & Unaligned Checkpoint 
> enabled)
>            Reporter: Tan Kim
>            Assignee: Stefan Richter
>            Priority: Critical
>              Labels: stability
>         Attachments: image-2023-04-29-02-49-05-607.png, jobmanager_error.txt, 
> taskmanager_error.txt
>
>
> I'm testing Autoscaler through Kubernetes Operator and I'm facing the 
> following issue.
> As you know, when a job is scaled down through the autoscaler, the job 
> manager and task manager go down and then back up again.
> When this happens, an index out of bounds exception is thrown and the state 
> is not restored from a checkpoint.
> [~gyfora] told me via the Flink Slack troubleshooting channel that this is 
> likely an issue with Unaligned Checkpoint and not an issue with the 
> autoscaler, but I'm opening a ticket with Gyula for more clarification.
> Please see the attached JM and TM error logs.
> Thank you.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to