Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-16873

force spill NPE

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.0.0
    • 1.6.3, 2.0.1, 2.1.0
    • Spark Core
    • None

    Description

      16/07/31 20:52:38 INFO executor.Executor: Running task 1090.1 in stage 18.0 (TID 23793)
      16/07/31 20:52:38 INFO storage.ShuffleBlockFetcherIterator: Getting 5000 non-empty blocks out of 5000 blocks
      16/07/31 20:52:38 INFO collection.ExternalSorter: Thread 151 spilling in-memory map of 410.3 MB to disk (1 time so far)
      16/07/31 20:52:38 INFO storage.ShuffleBlockFetcherIterator: Started 30 remote fetches in 214 ms
      16/07/31 20:52:44 INFO collection.ExternalSorter: spill memory to

      file:/data5/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-e9cc29b9-ca1a-460a-ad76-

      32f8ee437e51/0d/temp_shuffle_dc8f4489-2289-4b99-a605-81c873dc9e17, fileSize:46.0 MB
      16/07/31 20:52:44 WARN memory.ExecutionMemoryPool: Internal error: release called on 430168384 bytes but task only has

      424958272 bytes of memory from the on-heap execution pool
      16/07/31 20:52:45 INFO collection.ExternalSorter: Thread 152 spilling in-memory map of 398.7 MB to disk (1 time so far)
      16/07/31 20:52:52 INFO collection.ExternalSorter: Thread 151 spilling in-memory map of 389.6 MB to disk (2 times so far)
      16/07/31 20:52:54 INFO collection.ExternalSorter: spill memory to

      file:/data11/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-56e1ec4d-9560-4019-83f7-

      ec6fd3fe78f9/24/temp_shuffle_10a210a4-38df-40e6-821d-00e15da12eaa, fileSize:45.5 MB
      16/07/31 20:52:54 WARN memory.ExecutionMemoryPool: Internal error: release called on 413772021 bytes but task only has

      408561909 bytes of memory from the on-heap execution pool
      16/07/31 20:52:55 INFO collection.ExternalSorter: spill memory to

      file:/data2/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-f4fe32a2-c930-4b49-8feb-

      722536c290d7/10/temp_shuffle_4ae9dd49-6039-4a44-901a-ee4650351ebc, fileSize:44.7 MB
      16/07/31 20:52:56 INFO collection.ExternalSorter: Thread 152 spilling in-memory map of 389.6 MB to disk (2 times so far)
      16/07/31 20:53:30 INFO collection.ExternalSorter: spill memory to

      file:/data6/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-95b49f89-4155-435c-826b-

      7a616662d47a/1b/temp_shuffle_ea43939f-04d4-42a9-805f-44e01afc9a13, fileSize:44.7 MB
      16/07/31 20:53:30 INFO collection.ExternalSorter: Thread 151 spilling in-memory map of 389.6 MB to disk (3 times so far)
      16/07/31 20:53:49 INFO collection.ExternalSorter: spill memory to

      file:/data10/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-4f736364-e192-4f31-bfe9-

      8eaa29ff2114/3f/temp_shuffle_680e07ec-404e-4710-9d14-1665b300e05c, fileSize:44.7 MB
      16/07/31 20:54:04 INFO collection.ExternalSorter: Task 23585 force spilling in-memory map to disk and it will release 164.3

      MB memory
      16/07/31 20:54:04 INFO collection.ExternalSorter: spill memory to

      file:/data4/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-db5f46c3-d7a4-4f93-8b77-

      565e469696fb/09/temp_shuffle_ec3ece08-4569-4197-893a-4a5dfcbbf9fa, fileSize:0.0 B
      16/07/31 20:54:04 WARN memory.TaskMemoryManager: leak 164.3 MB memory from

      org.apache.spark.util.collection.ExternalSorter@3db4b52d
      16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 23585
      16/07/31 20:54:04 ERROR executor.Executor: Exception in task 1013.0 in stage 18.0 (TID 23585)
      java.lang.NullPointerException
      at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:625)
      at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:540)
      at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:508)
      at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:814)
      at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:254)
      at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:109)
      at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:154)
      at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
      at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
      at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:346)
      at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:367)
      at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
      at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
      at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
      at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
      at org.apache.spark.scheduler.Task.run(Task.scala:89)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:227)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      at java.lang.Thread.run(Thread.java:744)
      16/07/31 20:54:30 INFO executor.Executor: Executor is trying to kill task 1090.1 in stage 18.0 (TID 23793)
      16/07/31 20:54:30 INFO executor.CoarseGrainedExecutorBackend: Driver commanded a shutdown
      16/07/31 20:54:31 INFO storage.MemoryStore: MemoryStore cleared
      16/07/31 20:54:31 INFO storage.BlockManager: BlockManager stopped
      16/07/31 20:54:31 INFO remote.RemoteActorRefProvider$RemotingTerminator: Shutting down remote daemon.
      16/07/31 20:54:31 INFO remote.RemoteActorRefProvider$RemotingTerminator: Remote daemon shut down; proceeding with flushing

      remote transports.
      16/07/31 20:54:31 INFO remote.RemoteActorRefProvider$RemotingTerminator: Remoting shut down.
      16/07/31 20:54:31 INFO util.ShutdownHookManager: Shutdown hook called

      Attachments

        Issue Links

          Activity

            People

              sharkd sharkd tu
              sharkd sharkd tu
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: