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

Managed memory may not be released in time when operators use managed memory frequently

    XMLWordPrintableJSON

Details

    Description

      Some batch operators (like sort merge join or hash aggregate) use managed memory frequently. When these operators are chained together and the cluster load is a bit heavy, it is very likely that the following exception occurs:

      2020-12-18 10:04:32
      java.lang.RuntimeException: org.apache.flink.runtime.memory.MemoryAllocationException: Could not allocate 512 pages
      	at org.apache.flink.table.runtime.util.LazyMemorySegmentPool.nextSegment(LazyMemorySegmentPool.java:85)
      	at org.apache.flink.runtime.io.disk.SimpleCollectingOutputView.<init>(SimpleCollectingOutputView.java:49)
      	at org.apache.flink.table.runtime.operators.aggregate.BytesHashMap$RecordArea.<init>(BytesHashMap.java:297)
      	at org.apache.flink.table.runtime.operators.aggregate.BytesHashMap.<init>(BytesHashMap.java:103)
      	at org.apache.flink.table.runtime.operators.aggregate.BytesHashMap.<init>(BytesHashMap.java:90)
      	at LocalHashAggregateWithKeys$209161.open(Unknown Source)
      	at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:401)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$2(StreamTask.java:506)
      	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:92)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:501)
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:530)
      	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722)
      	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547)
      	at java.lang.Thread.run(Thread.java:834)
      	Suppressed: java.lang.NullPointerException
      		at LocalHashAggregateWithKeys$209161.close(Unknown Source)
      		at org.apache.flink.table.runtime.operators.TableStreamOperator.dispose(TableStreamOperator.java:46)
      		at org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:739)
      		at org.apache.flink.streaming.runtime.tasks.StreamTask.runAndSuppressThrowable(StreamTask.java:719)
      		at org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUpInvoke(StreamTask.java:642)
      		at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:551)
      		... 3 more
      		Suppressed: java.lang.NullPointerException
      			at LocalHashAggregateWithKeys$209766.close(Unknown Source)
      			... 8 more
      Caused by: org.apache.flink.runtime.memory.MemoryAllocationException: Could not allocate 512 pages
      	at org.apache.flink.runtime.memory.MemoryManager.allocatePages(MemoryManager.java:231)
      	at org.apache.flink.table.runtime.util.LazyMemorySegmentPool.nextSegment(LazyMemorySegmentPool.java:83)
      	... 13 more
      Caused by: org.apache.flink.runtime.memory.MemoryReservationException: Could not allocate 16777216 bytes, only 9961487 bytes are remaining. This usually indicates that you are requesting more memory than you have reserved. However, when running an old JVM version it can also be caused by slow garbage collection. Try to upgrade to Java 8u72 or higher if running on an old Java version.
      	at org.apache.flink.runtime.memory.UnsafeMemoryBudget.reserveMemory(UnsafeMemoryBudget.java:164)
      	at org.apache.flink.runtime.memory.UnsafeMemoryBudget.reserveMemory(UnsafeMemoryBudget.java:80)
      	at org.apache.flink.runtime.memory.MemoryManager.allocatePages(MemoryManager.java:229)
      	... 14 more
      

      It seems that this is caused by relying on GC to release managed memory, as System.gc() may not trigger GC in time. See UnsafeMemoryBudget.java.

      Attachments

        1. summary.py
          9 kB
          zhou
        2. taskmanager.log
          440 kB
          zhou
        3. exception
          548 kB
          zhou

        Issue Links

          Activity

            People

              xtsong Xintong Song
              TsReaper Caizhi Weng
              Votes:
              0 Vote for this issue
              Watchers:
              12 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: