-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-16873] [Core] Fix SpillReader NPE when spillFile has no data #14479
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Conversation
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Jenkins test this please |
LGTM |
Test build #63171 has finished for PR 14479 at commit
|
asfgit
pushed a commit
that referenced
this pull request
Aug 4, 2016
## What changes were proposed in this pull request? SpillReader NPE when spillFile has no data. See follow logs: 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.ExternalSorter3db4b52d 16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 2358516/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:624) at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539) at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:507) at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816) at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251) 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 ## How was this patch tested? Manual test. Author: sharkd <sharkd.tu@gmail.com> Author: sharkdtu <sharkdtu@tencent.com> Closes #14479 from sharkdtu/master. (cherry picked from commit 583d91a) Signed-off-by: Reynold Xin <rxin@databricks.com>
LGTM - merging in master/2.0/1.6. |
asfgit
pushed a commit
that referenced
this pull request
Aug 4, 2016
## What changes were proposed in this pull request? SpillReader NPE when spillFile has no data. See follow logs: 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.ExternalSorter3db4b52d 16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 2358516/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:624) at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539) at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:507) at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816) at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251) 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 ## How was this patch tested? Manual test. Author: sharkd <sharkd.tu@gmail.com> Author: sharkdtu <sharkdtu@tencent.com> Closes #14479 from sharkdtu/master. (cherry picked from commit 583d91a) Signed-off-by: Reynold Xin <rxin@databricks.com>
zzcclp
pushed a commit
to zzcclp/spark
that referenced
this pull request
Aug 4, 2016
## What changes were proposed in this pull request? SpillReader NPE when spillFile has no data. See follow logs: 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.ExternalSorter3db4b52d 16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 2358516/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:624) at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539) at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:507) at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816) at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251) 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 ## How was this patch tested? Manual test. Author: sharkd <sharkd.tu@gmail.com> Author: sharkdtu <sharkdtu@tencent.com> Closes apache#14479 from sharkdtu/master. (cherry picked from commit 583d91a) Signed-off-by: Reynold Xin <rxin@databricks.com> (cherry picked from commit d2518ac)
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
SpillReader NPE when spillFile has no data. See follow logs:
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 = 2358516/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:624)
at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539)
at org.apache.spark.util.collection.ExternalSorter$SpillReader.(ExternalSorter.scala:507)
at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816)
at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251)
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
How was this patch tested?
Manual test.