Nico Kruber created FLINK-20099:
-----------------------------------

             Summary: HeapStateBackend checkpoint error hidden under cryptic 
message
                 Key: FLINK-20099
                 URL: https://issues.apache.org/jira/browse/FLINK-20099
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Checkpointing, Runtime / State Backends
    Affects Versions: 1.11.2
            Reporter: Nico Kruber
         Attachments: Screenshot_20201112_001331.png

When the memory state back-end hits a certain size, it fails to permit 
checkpoints. Even though a very detailed exception is thrown at its source, 
this is neither logged nor shown in the UI:
 * Logs just contain:

{code:java}
00:06:41.462 [jobmanager-future-thread-14] INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Decline checkpoint 
2 by task 8eb303cd3196310cb2671212f4ed013c of job 
c9b7a410bd3143864ca23ba89595d878 at 6a73bcf2-46b6-4735-a616-fdf09ff1471c @ 
localhost (dataPort=-1).
{code}
 * UI: (also see the attached Screenshot_20201112_001331.png)

{code:java}
Failure Message: The job has failed.
{code}
-> this isn't even true: the job is still running fine!

 

Debugging into {{PendingCheckpoint#abort()}} reveals that the causing exception 
is actually still in there but the detailed information from it is just never 
used.
 For reference, this is what is available there and should be logged or shown:
{code:java}
java.lang.Exception: Could not materialize checkpoint 2 for operator aggregates 
-> (Sink: sink-agg-365, Sink: sink-agg-180, Sink: sink-agg-45, Sink: 
sink-agg-30) (4/4).
        at 
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.handleExecutionException(AsyncCheckpointRunnable.java:191)
        at 
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:138)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Size 
of the state is larger than the maximum permitted memory-backed state. 
Size=6122737 , maxSize=5242880 . Consider using a different state backend, like 
the File System State backend.
        at java.util.concurrent.FutureTask.report(FutureTask.java:122)
        at java.util.concurrent.FutureTask.get(FutureTask.java:192)
        at 
org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:479)
        at 
org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:50)
        at 
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:102)
        ... 3 more
Caused by: java.io.IOException: Size of the state is larger than the maximum 
permitted memory-backed state. Size=6122737 , maxSize=5242880 . Consider using 
a different state backend, like the File System State backend.
        at 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.checkSize(MemCheckpointStreamFactory.java:64)
        at 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetBytes(MemCheckpointStreamFactory.java:145)
        at 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetHandle(MemCheckpointStreamFactory.java:126)
        at 
org.apache.flink.runtime.state.CheckpointStreamWithResultProvider$PrimaryStreamOnly.closeAndFinalizeCheckpointStreamResult(CheckpointStreamWithResultProvider.java:77)
        at 
org.apache.flink.runtime.state.heap.HeapSnapshotStrategy$1.callInternal(HeapSnapshotStrategy.java:199)
        at 
org.apache.flink.runtime.state.heap.HeapSnapshotStrategy$1.callInternal(HeapSnapshotStrategy.java:158)
        at 
org.apache.flink.runtime.state.AsyncSnapshotCallable.call(AsyncSnapshotCallable.java:75)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
org.apache.flink.runtime.concurrent.FutureUtils.runIfNotDoneAndGet(FutureUtils.java:476)
        ... 5 more
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to