Github user mridulm commented on the pull request: https://github.com/apache/spark/pull/1499#issuecomment-50115353 Running tests with export SPARK_JAVA_OPTS="-Dspark.shuffle.manager=org.apache.spark.shuffle.sort.SortShuffleManager" causes : ''' - sorting using mutable pairs *** FAILED *** org.apache.spark.SparkException: Job aborted due to stage failure: Task 3.0:1 failed 4 times, most recent failure: Exception failure in TID 14 on host localhost: java.lang.ArrayStoreException: scala.Tuple2 scala.runtime.ScalaRunTime$.array_update(ScalaRunTime.scala:88) scala.Array$.slowcopy(Array.scala:81) scala.Array$.copy(Array.scala:107) scala.collection.mutable.ResizableArray$class.copyToArray(ResizableArray.scala:77) scala.collection.mutable.ArrayBuffer.copyToArray(ArrayBuffer.scala:47) scala.collection.TraversableOnce$class.copyToArray(TraversableOnce.scala:241) scala.collection.AbstractTraversable.copyToArray(Traversable.scala:105) scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:249) scala.collection.AbstractTraversable.toArray(Traversable.scala:105) scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) org.apache.spark.InterruptibleIterator.toArray(InterruptibleIterator.scala:28) org.apache.spark.rdd.OrderedRDDFunctions$$anonfun$sortByKey$1.apply(OrderedRDDFunctions.scala:62) org.apache.spark.rdd.OrderedRDDFunctions$$anonfun$sortByKey$1.apply(OrderedRDDFunctions.scala:61) org.apache.spark.rdd.RDD$$anonfun$12.apply(RDD.scala:581) org.apache.spark.rdd.RDD$$anonfun$12.apply(RDD.scala:581) org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) org.apache.spark.rdd.RDD.iterator(RDD.scala:229) org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:112) org.apache.spark.scheduler.Task.run(Task.scala:51) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1039) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1037) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1037) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:641) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:641) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:641) ... ''' The actual classes are : ''' classes = <[class scala.Tuple2] ; [class scala.Tuple2, class scala.Tuple2, class scala.Tuple2]> ''' obtained using : ''' def collect(): Array[T] = { println("classes = " + sc.runJob(this, (iter: Iterator[T]) => iter.map(v => if (v == null) "null" else v.getClass).mkString("[", ",\n\t\t", "]")). mkString("<", " ; \n\t", ">")) System.out.flush() val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray) ''' in RDD.collect
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---