[ 
https://issues.apache.org/jira/browse/SYSTEMML-946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15512016#comment-15512016
 ] 

Mike Dusenberry commented on SYSTEMML-946:
------------------------------------------

Latest run with the updates resulted in a few more OOM errors:

Heap space:
{code}
java.lang.OutOfMemoryError: Java heap space
        at 
org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:755)
        at 
org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:494)
        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:127)
        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:208)
        at 
org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201)
        at 
org.apache.spark.rdd.SqlNewHadoopRDD$$anon$1.hasNext(SqlNewHadoopRDD.scala:194)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
        at 
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:29)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:928)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:894)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
        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:214)
        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:745)
{code}

GC:
{code}
java.lang.OutOfMemoryError: GC overhead limit exceeded
        at java.util.Arrays.copyOf(Arrays.java:3380)
        at 
org.apache.sysml.runtime.matrix.data.SparseRow.recap(SparseRow.java:157)
        at 
org.apache.sysml.runtime.matrix.data.SparseRow.append(SparseRow.java:243)
        at 
org.apache.sysml.runtime.matrix.data.SparseBlockMCSR.append(SparseBlockMCSR.java:243)
        at 
org.apache.sysml.runtime.matrix.data.MatrixBlock.appendValue(MatrixBlock.java:714)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:951)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:894)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
        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:214)
        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:745)
{code}
{code}
java.lang.OutOfMemoryError: GC overhead limit exceeded
        at java.lang.Double.valueOf(Double.java:519)
        at scala.runtime.BoxesRunTime.boxToDouble(BoxesRunTime.java:84)
        at 
org.apache.spark.sql.execution.datasources.parquet.ParentContainerUpdater$class.setDouble(CatalystRowConverter.scala:59)
        at 
org.apache.spark.sql.execution.datasources.parquet.CatalystRowConverter$CatalystArrayConverter$ElementConverter$$anon$7.setDouble(CatalystRowConverter.scala:501)
        at 
org.apache.spark.sql.execution.datasources.parquet.CatalystPrimitiveConverter.addDouble(CatalystRowConverter.scala:87)
        at 
org.apache.parquet.column.impl.ColumnReaderImpl$2$2.writeValue(ColumnReaderImpl.java:229)
        at 
org.apache.parquet.column.impl.ColumnReaderImpl.writeCurrentValueToConverter(ColumnReaderImpl.java:365)
        at 
org.apache.parquet.io.RecordReaderImplementation.read(RecordReaderImplementation.java:405)
        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:209)
        at 
org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201)
        at 
org.apache.spark.rdd.SqlNewHadoopRDD$$anon$1.hasNext(SqlNewHadoopRDD.scala:194)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
        at 
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:29)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:928)
        at 
org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils$DataFrameToBinaryBlockFunction.call(RDDConverterUtils.java:894)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:192)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
        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:214)
        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:745)
{code}

> OOM on spark dataframe-matrix / csv-matrix conversion
> -----------------------------------------------------
>
>                 Key: SYSTEMML-946
>                 URL: https://issues.apache.org/jira/browse/SYSTEMML-946
>             Project: SystemML
>          Issue Type: Bug
>          Components: Runtime
>            Reporter: Matthias Boehm
>         Attachments: mnist_lenet.dml
>
>
> The decision on dense/sparse block allocation in our dataframeToBinaryBlock 
> and csvToBinaryBlock data converters is purely based on the sparsity. This 
> works very well for the common case of tall & skinny matrices. However, for 
> scenarios with dense data but huge number of columns a single partition will 
> rarely have 1000 rows to fill an entire row of blocks. This leads to 
> unnecessary allocation and dense-sparse conversion as well as potential 
> out-of-memory errors because the temporary memory requirement can be up to 
> 1000x larger than the input partition.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to