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

Dapeng Sun commented on HIVE-15580:
-----------------------------------

Thank [~xuefuz], [~csun] and [~Ferd], we are running a 100TB test case about 
data skew on 50 nodes(TPC-xBB q21), before the patch, spark tasks are failed 
with following error:
{noformat}
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
         at java.util.Arrays.copyOf(Arrays.java:3181)
         at java.util.ArrayList.grow(ArrayList.java:261)
         at java.util.ArrayList.ensureExplicitCapacity(ArrayList.java:235)
         at java.util.ArrayList.ensureCapacityInternal(ArrayList.java:227)
         at java.util.ArrayList.add(ArrayList.java:458)
         at 
org.apache.hadoop.hive.ql.exec.spark.SortByShuffler$ShuffleFunction$1.next(SortByShuffler.java:100)
         at 
org.apache.hadoop.hive.ql.exec.spark.SortByShuffler$ShuffleFunction$1.next(SortByShuffler.java:75)
         at 
org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList$ResultIterator.hasNext(HiveBaseFunctionResultList.java:95)
         at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
         at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:200)
         at 
org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:64)
         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)
{noformat}
after apply the patch, the arraylist are fixed, but PartitionedPairBuffer also 
cause OOM, here are the task failed exception:
{noformat}
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
        at 
org.apache.spark.util.collection.PartitionedPairBuffer.growArray(PartitionedPairBuffer.scala:67)
        at 
org.apache.spark.util.collection.PartitionedPairBuffer.insert(PartitionedPairBuffer.scala:48)
        at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:203)
        at 
org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:111)
        at org.apache.spark.rdd.ShuffledRDD.compute(ShuffledRDD.scala:98)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
        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.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)
{noformat}

> Replace Spark's groupByKey operator with something with bounded memory
> ----------------------------------------------------------------------
>
>                 Key: HIVE-15580
>                 URL: https://issues.apache.org/jira/browse/HIVE-15580
>             Project: Hive
>          Issue Type: Improvement
>          Components: Spark
>            Reporter: Xuefu Zhang
>            Assignee: Xuefu Zhang
>         Attachments: HIVE-15580.1.patch, HIVE-15580.1.patch, 
> HIVE-15580.2.patch, HIVE-15580.2.patch, HIVE-15580.3.patch, HIVE-15580.patch
>
>




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

Reply via email to