You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@systemml.apache.org by "Fei Hu (JIRA)" <ji...@apache.org> on 2017/07/12 19:10:00 UTC
[jira] [Commented] (SYSTEMML-1762) Improve the robustness of sparse
matrix reshape function for the Spark mode
[ https://issues.apache.org/jira/browse/SYSTEMML-1762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16084521#comment-16084521 ]
Fei Hu commented on SYSTEMML-1762:
----------------------------------
The error messages are as following:
{code:java}
17/07/12 12:04:47 ERROR TaskSetManager: Task 1 in stage 177.0 failed 1 times; aborting job
17/07/12 12:04:47 INFO TaskSetManager: Lost task 3.0 in stage 177.0 (TID 528) on localhost, executor driver: java.lang.NullPointerException (null) [duplicate 1]
17/07/12 12:04:47 INFO TaskSchedulerImpl: Cancelling stage 177
17/07/12 12:04:47 INFO TaskSchedulerImpl: Stage 177 was cancelled
17/07/12 12:04:47 INFO Executor: Executor is trying to kill task 2.0 in stage 177.0 (TID 527)
17/07/12 12:04:47 INFO Executor: Executor is trying to kill task 0.0 in stage 177.0 (TID 525)
17/07/12 12:04:47 INFO DAGScheduler: ShuffleMapStage 177 (flatMapToPair at MatrixReshapeSPInstruction.java:106) failed in 0.016 s due to Job aborted due to stage failure: Task 1 in stage 177.0 failed 1 times, most recent failure: Lost task 1.0 in stage 177.0 (TID 526, localhost, executor driver): java.lang.NullPointerException
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshapeSparse(LibMatrixReorg.java:1591)
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshape(LibMatrixReorg.java:504)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:138)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:114)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:191)
at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Driver stacktrace:
17/07/12 12:04:47 INFO DAGScheduler: Job 139 failed: fold at RDDAggregateUtils.java:137, took 0.018972 s
17/07/12 12:04:47 INFO Executor: Executor killed task 0.0 in stage 177.0 (TID 525)
17/07/12 12:04:47 ERROR ParWorker: Failed to execute task (type=SET, iterations={[j=3]}), retry:0
org.apache.sysml.runtime.DMLRuntimeException: ERROR: Runtime error in program block generated from statement block between lines 0 and 0 -- Error evaluating instruction: SPARK°uark+°_mVar3618·MATRIX·DOUBLE°_mVar3619·MATRIX·DOUBLE°SINGLE_BLOCK
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeSingleInstruction(ProgramBlock.java:316)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeInstructions(ProgramBlock.java:217)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.execute(ProgramBlock.java:163)
at org.apache.sysml.runtime.controlprogram.parfor.ParWorker.executeSetTask(ParWorker.java:167)
at org.apache.sysml.runtime.controlprogram.parfor.ParWorker.executeTask(ParWorker.java:136)
at org.apache.sysml.runtime.controlprogram.parfor.LocalParWorker.run(LocalParWorker.java:122)
at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 177.0 failed 1 times, most recent failure: Lost task 1.0 in stage 177.0 (TID 526, localhost, executor driver): java.lang.NullPointerException
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshapeSparse(LibMatrixReorg.java:1591)
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshape(LibMatrixReorg.java:504)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:138)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:114)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:191)
at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1435)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1422)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at scala.Option.foreach(Option.scala:257)
at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1650)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1594)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1925)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1988)
at org.apache.spark.rdd.RDD$$anonfun$fold$1.apply(RDD.scala:1089)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
at org.apache.spark.rdd.RDD.withScope(RDD.scala:362)
at org.apache.spark.rdd.RDD.fold(RDD.scala:1083)
at org.apache.spark.api.java.JavaRDDLike$class.fold(JavaRDDLike.scala:414)
at org.apache.spark.api.java.AbstractJavaRDDLike.fold(JavaRDDLike.scala:45)
at org.apache.sysml.runtime.instructions.spark.utils.RDDAggregateUtils.aggStable(RDDAggregateUtils.java:137)
at org.apache.sysml.runtime.instructions.spark.AggregateUnarySPInstruction.processInstruction(AggregateUnarySPInstruction.java:102)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeSingleInstruction(ProgramBlock.java:286)
... 6 more
Caused by: java.lang.NullPointerException
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshapeSparse(LibMatrixReorg.java:1591)
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshape(LibMatrixReorg.java:504)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:138)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:114)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:191)
at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
... 1 more
17/07/12 12:04:47 INFO Executor: Executor killed task 2.0 in stage 177.0 (TID 527)
17/07/12 12:04:47 ERROR ParWorker: Error executing task:
org.apache.sysml.runtime.DMLRuntimeException: ERROR: Runtime error in program block generated from statement block between lines 0 and 0 -- Error evaluating instruction: SPARK°uark+°_mVar3618·MATRIX·DOUBLE°_mVar3619·MATRIX·DOUBLE°SINGLE_BLOCK
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeSingleInstruction(ProgramBlock.java:316)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeInstructions(ProgramBlock.java:217)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.execute(ProgramBlock.java:163)
at org.apache.sysml.runtime.controlprogram.parfor.ParWorker.executeSetTask(ParWorker.java:167)
at org.apache.sysml.runtime.controlprogram.parfor.ParWorker.executeTask(ParWorker.java:136)
at org.apache.sysml.runtime.controlprogram.parfor.LocalParWorker.run(LocalParWorker.java:122)
at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 177.0 failed 1 times, most recent failure: Lost task 1.0 in stage 177.0 (TID 526, localhost, executor driver): java.lang.NullPointerException
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshapeSparse(LibMatrixReorg.java:1591)
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshape(LibMatrixReorg.java:504)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:138)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:114)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:191)
at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1435)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1422)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at scala.Option.foreach(Option.scala:257)
at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1650)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1594)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1925)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1988)
at org.apache.spark.rdd.RDD$$anonfun$fold$1.apply(RDD.scala:1089)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
at org.apache.spark.rdd.RDD.withScope(RDD.scala:362)
at org.apache.spark.rdd.RDD.fold(RDD.scala:1083)
at org.apache.spark.api.java.JavaRDDLike$class.fold(JavaRDDLike.scala:414)
at org.apache.spark.api.java.AbstractJavaRDDLike.fold(JavaRDDLike.scala:45)
at org.apache.sysml.runtime.instructions.spark.utils.RDDAggregateUtils.aggStable(RDDAggregateUtils.java:137)
at org.apache.sysml.runtime.instructions.spark.AggregateUnarySPInstruction.processInstruction(AggregateUnarySPInstruction.java:102)
at org.apache.sysml.runtime.controlprogram.ProgramBlock.executeSingleInstruction(ProgramBlock.java:286)
... 6 more
Caused by: java.lang.NullPointerException
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshapeSparse(LibMatrixReorg.java:1591)
at org.apache.sysml.runtime.matrix.data.LibMatrixReorg.reshape(LibMatrixReorg.java:504)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:138)
at org.apache.sysml.runtime.instructions.spark.MatrixReshapeSPInstruction$RDDReshapeFunction.call(MatrixReshapeSPInstruction.java:114)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$3$1.apply(JavaRDDLike.scala:143)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:191)
at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
... 1 more
17/07/12 12:04:47 ERROR ParWorker: Stopping LocalParWorker.
{code}
> Improve the robustness of sparse matrix reshape function for the Spark mode
> ---------------------------------------------------------------------------
>
> Key: SYSTEMML-1762
> URL: https://issues.apache.org/jira/browse/SYSTEMML-1762
> Project: SystemML
> Issue Type: Bug
> Components: Algorithms, ParFor, Runtime
> Reporter: Fei Hu
> Attachments: MNIST_Distrib_Sgd.scala
>
>
> When running the [distributed MNIST LeNet example | https://github.com/apache/systemml/blob/master/scripts/nn/examples/mnist_lenet_distrib_sgd.dml], there is a
> {{java.lang.NullPointerException}} error when reshaping the sparse matrix. The involved function is {{org.apache.sysml.runtime.matrix.data.LibMatrixReorg#reshapeSparse}} . The reason is that the output matrix index computed by {{org.apache.sysml.runtime.matrix.data.LibMatrixReorg#computeResultBlockIndex}} does not exist in the {{HashMap<MatrixIndexes,MatrixBlock> rix}}.
> To reproduce the error, the attached scala file {{MNIST_Distrib_Sgd.scala}} could be used to run the distributed MNIST example.
> If adding some codes to ignore the null output matrix block from {{MatrixBlock out = rix.get(ixtmp)}}, the distributed MNIST example could run in the Spark mode, but the result may not be right.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)