Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-3032

Potential bug when running sort-based shuffle with sorting using TimSort

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.1.0
    • 1.1.1, 1.2.0
    • Shuffle, Spark Core
    • None

    Description

      When using SparkPerf's aggregate-by-key workload to test sort-based shuffle, data type for key and value is (String, String), always meet this issue:

      java.lang.IllegalArgumentException: Comparison method violates its general contract!
              at org.apache.spark.util.collection.Sorter$SortState.mergeLo(Sorter.java:755)
              at org.apache.spark.util.collection.Sorter$SortState.mergeAt(Sorter.java:493)
              at org.apache.spark.util.collection.Sorter$SortState.mergeCollapse(Sorter.java:420)
              at org.apache.spark.util.collection.Sorter$SortState.access$200(Sorter.java:294)
              at org.apache.spark.util.collection.Sorter.sort(Sorter.java:128)
              at org.apache.spark.util.collection.SizeTrackingPairBuffer.destructiveSortedIterator(SizeTrackingPairBuffer.scala:83)
              at org.apache.spark.util.collection.ExternalSorter.spillToMergeableFile(ExternalSorter.scala:323)
              at org.apache.spark.util.collection.ExternalSorter.spill(ExternalSorter.scala:271)
              at org.apache.spark.util.collection.ExternalSorter.maybeSpill(ExternalSorter.scala:249)
              at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:220)
              at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:85)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
              at org.apache.spark.scheduler.Task.run(Task.scala:54)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
              at java.lang.Thread.run(Thread.java:722)
      

      Seems the current partitionKeyComparator which use hashcode of String as key comparator break some sorting contracts.

      Also I tested using data type Int as key, this is OK to pass the test, since hashcode of Int is its self. So I think potentially partitionDiff + hashcode of String may break the sorting contracts.

      Attachments

        Issue Links

          Activity

            People

              jerryshao Saisai Shao
              jerryshao Saisai Shao
              Votes:
              1 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: