Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-20099

HeapStateBackend checkpoint error hidden under cryptic message

    XMLWordPrintableJSON

Details

    Description

      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:
      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).
      
      • UI: (also see the attached Screenshot_20201112_001331.png)
      Failure Message: The job has failed.
      

      -> 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:

      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
      

      Attachments

        1. Screenshot_20201112_001331.png
          33 kB
          Nico Kruber

        Issue Links

          Activity

            People

              Unassigned Unassigned
              nkruber Nico Kruber
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: