Skip to content

Commit

Permalink
Revise comments; add handler to guard against future ShuffleManager i…
Browse files Browse the repository at this point in the history
…mplementations
  • Loading branch information
JoshRosen committed Jun 13, 2015
1 parent 07bb2c9 commit 7a14129
Showing 1 changed file with 10 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner, SparkEn
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.{RDD, ShuffledRDD}
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.hash.HashShuffleManager
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.shuffle.unsafe.UnsafeShuffleManager
import org.apache.spark.sql.catalyst.errors.attachTree
Expand Down Expand Up @@ -92,8 +93,11 @@ case class Exchange(
} else if (serializeMapOutputs && serializer.supportsRelocationOfSerializedObjects) {
// SPARK-4550 extended sort-based shuffle to serialize individual records prior to sorting
// them. This optimization is guarded by a feature-flag and is only applied in cases where
// shuffle dependency does not specify an aggregator and the record serializer has certain
// properties. If this optimization is enabled, we can safely avoid the copy.
// shuffle dependency does not specify an aggregator or ordering and the record serializer
// has certain properties. If this optimization is enabled, we can safely avoid the copy.
//
// Exchange never configures its ShuffledRDDs with aggregators or key orderings, so we only
// need to check whether the optimization is enabled and supported by our serializer.
//
// This optimization also applies to UnsafeShuffleManager (added in SPARK-7081).
false
Expand All @@ -104,9 +108,12 @@ case class Exchange(
// both cases, we must copy.
true
}
} else {
} else if (shuffleManager.isInstanceOf[HashShuffleManager]) {
// We're using hash-based shuffle, so we don't need to copy.
false
} else {
// Catch-all case to safely handle any future ShuffleManager implementations.
true
}
}

Expand Down

0 comments on commit 7a14129

Please sign in to comment.