I haven't done a comparative benchmarking between the two, and it would involve some work to do so. A single run with each suffler would probably not say that much since we have a rather busy cluster and the performance heavily depends on what's currently running in the cluster. I have seen less problems/exceptions though, and possibilities to decrease memory requirements (or increase cores), which is of great help.
BTW, is it possible (or will it be) to use Tungsten with dynamic allocation and the external shuffle manager? Best, Anders On Tue, Sep 1, 2015 at 7:07 PM Davies Liu <dav...@databricks.com> wrote: > Thanks for the confirmation. The tungsten-sort is not the default > ShuffleManager, this fix will not block 1.5 release, it may be in > 1.5.1. > > BTW, How is the difference between sort and tungsten-sort > ShuffleManager for this large job? > > On Tue, Sep 1, 2015 at 8:03 AM, Anders Arpteg <arp...@spotify.com> wrote: > > A fix submitted less than one hour after my mail, very impressive Davies! > > I've compiled your PR and tested it with the large job that failed > before, > > and it seems to work fine now without any exceptions. Awesome, thanks! > > > > Best, > > Anders > > > > On Tue, Sep 1, 2015 at 1:38 AM Davies Liu <dav...@databricks.com> wrote: > >> > >> I had sent out a PR [1] to fix 2), could you help to test that? > >> > >> [1] https://github.com/apache/spark/pull/8543 > >> > >> On Mon, Aug 31, 2015 at 12:34 PM, Anders Arpteg <arp...@spotify.com> > >> wrote: > >> > Was trying out 1.5 rc2 and noticed some issues with the Tungsten > shuffle > >> > manager. One problem was when using the com.databricks.spark.avro > reader > >> > and > >> > the error(1) was received, see stack trace below. The problem does not > >> > occur > >> > with the "sort" shuffle manager. > >> > > >> > Another problem was in a large complex job with lots of > transformations > >> > occurring simultaneously, i.e. 50+ or more maps each shuffling data. > >> > Received error(2) about inability to acquire memory which seems to > also > >> > have > >> > to do with Tungsten. Possibly some setting available to increase that > >> > memory, because there's lots of heap memory available. > >> > > >> > Am running on Yarn 2.2 with about 400 executors. Hoping this will give > >> > some > >> > hints for improving the upcoming release, or for me to get some hints > to > >> > fix > >> > the problems. > >> > > >> > Thanks, > >> > Anders > >> > > >> > Error(1) > >> > > >> > 15/08/31 18:30:57 WARN TaskSetManager: Lost task 0.0 in stage 1.0 (TID > >> > 3387, > >> > lon4-hadoopslave-c245.lon4.spotify.net): java.io.EOFException > >> > > >> > at java.io.DataInputStream.readInt(DataInputStream.java:392) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$3$$anon$1.next(UnsafeRowSerializer.scala:121) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$3$$anon$1.next(UnsafeRowSerializer.scala:109) > >> > > >> > at scala.collection.Iterator$$anon$13.next(Iterator.scala:372) > >> > > >> > at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) > >> > > >> > at > >> > > >> > > org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30) > >> > > >> > at > >> > > >> > > org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43) > >> > > >> > at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.processInputs(TungstenAggregationIterator.scala:366) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.start(TungstenAggregationIterator.scala:622) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$ > 1.org$apache$spark$sql$execution$aggregate$Tung > >> > > >> > > stenAggregate$$anonfun$$executePartition$1(TungstenAggregate.scala:110) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119) > >> > > >> > at > >> > > >> > > org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119) > >> > > >> > at > >> > > >> > > org.apache.spark.rdd.MapPartitionsWithPreparationRDD.compute(MapPartitionsWithPreparationRDD.scala:47) > >> > > >> > at > >> > org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297) > >> > > >> > at org.apache.spark.rdd.RDD.iterator(RDD.scala:264) > >> > > >> > at > >> > > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > >> > > >> > at > >> > org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297) > >> > > >> > at org.apache.spark.rdd.RDD.iterator(RDD.scala:264) > >> > > >> > at > >> > org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) > >> > > >> > at org.apache.spark.scheduler.Task.run(Task.scala:88) > >> > > >> > at > >> > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > >> > > >> > 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:745) > >> > > >> > > >> > Error(2) > >> > > >> > 5/08/31 18:41:25 WARN TaskSetManager: Lost task 16.1 in stage 316.0 > (TID > >> > 32686, lon4-hadoopslave-b925.lon4.spotify.net): java.io.IOException: > >> > Unable > >> > to acquire 67108864 bytes of memory > >> > > >> > at > >> > > >> > > org.apache.spark.shuffle.unsafe.UnsafeShuffleExternalSorter.acquireNewPageIfNecessary(UnsafeShuffleExternalSorter.java:385) > >> > > >> > at > >> > > >> > > org.apache.spark.shuffle.unsafe.UnsafeShuffleExternalSorter.insertRecord(UnsafeShuffleExternalSorter.java:435) > >> > > >> > at > >> > > >> > > org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:246) > >> > > >> > at > >> > > >> > > org.apache.spark.shuffle.unsafe.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:174) > >> > > >> > 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:88) > >> > > >> > at > >> > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > >> > > >> > 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:745) >