Skip to content

Commit 6d07678

Browse files
committed
Address comments.
1 parent 4273b8c commit 6d07678

File tree

2 files changed

+30
-9
lines changed

2 files changed

+30
-9
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala

Lines changed: 18 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -85,17 +85,28 @@ case class Exchange(
8585
keySchema: Array[DataType],
8686
valueSchema: Array[DataType],
8787
numPartitions: Int): Serializer = {
88+
// In ExternalSorter's spillToMergeableFile function, key-value pairs are written out
89+
// through write(key) and then write(value) instead of write((key, value)). Because
90+
// SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use
91+
// it when spillToMergeableFile in ExternalSorter will be used.
92+
// So, we will not use SparkSqlSerializer2 when
93+
// - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater
94+
// then the bypassMergeThreshold; or
95+
// - newOrdering is defined.
96+
val cannotUseSqlSerializer2 =
97+
(sortBasedShuffleOn && numPartitions > bypassMergeThreshold) || newOrdering.nonEmpty
98+
8899
val useSqlSerializer2 =
89-
!(sortBasedShuffleOn && numPartitions > bypassMergeThreshold) &&
90-
child.sqlContext.conf.useSqlSerializer2 &&
91-
SparkSqlSerializer2.support(keySchema) &&
92-
SparkSqlSerializer2.support(valueSchema)
100+
child.sqlContext.conf.useSqlSerializer2 && // SparkSqlSerializer2 is enabled.
101+
!cannotUseSqlSerializer2 && // Safe to use Serializer2.
102+
SparkSqlSerializer2.support(keySchema) && // The schema of key is supported.
103+
SparkSqlSerializer2.support(valueSchema) // The schema of value is supported.
93104

94105
val serializer = if (useSqlSerializer2) {
95-
logInfo("Use SparkSqlSerializer2.")
106+
logInfo("Using SparkSqlSerializer2.")
96107
new SparkSqlSerializer2(keySchema, valueSchema)
97108
} else {
98-
logInfo("Use SparkSqlSerializer.")
109+
logInfo("Using SparkSqlSerializer.")
99110
new SparkSqlSerializer(sparkConf)
100111
}
101112

@@ -160,7 +171,7 @@ case class Exchange(
160171
} else {
161172
new ShuffledRDD[Row, Null, Null](rdd, part)
162173
}
163-
val keySchema = sortingExpressions.map(_.dataType).toArray
174+
val keySchema = child.output.map(_.dataType).toArray
164175
shuffled.setSerializer(serializer(keySchema, null, numPartitions))
165176

166177
shuffled.map(_._1)

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,17 @@ import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow
3131
import org.apache.spark.sql.types._
3232

3333
/**
34-
* The serialization stream for SparkSqlSerializer2.
34+
* The serialization stream for [[SparkSqlSerializer2]]. It assumes that the object passed in
35+
* its `writeObject` are [[Product2]]. The serialization functions for the key and value of the
36+
* [[Product2]] are constructed based on their schemata.
37+
* The benefit of this serialization stream is that compared with general-purpose serializers like
38+
* Kryo and Java serializer, it can significantly reduce the size of serialized and has a lower
39+
* allocation cost, which can benefit the shuffle operation. Right now, its main limitations are:
40+
* 1. It does not support complex types, i.e. Map, Array, and Struct.
41+
* 2. It assumes that the objects passed in are [[Product2]]. So, it cannot be used when
42+
* [[org.apache.spark.util.collection.ExternalSorter]]'s merge sort operation is used because
43+
* the objects passed in the serializer are not in the type of [[Product2]]. Also also see
44+
* the comment of the `serializer` method in [[Exchange]] for more information on it.
3545
*/
3646
private[sql] class Serializer2SerializationStream(
3747
keySchema: Array[DataType],
@@ -61,7 +71,7 @@ private[sql] class Serializer2SerializationStream(
6171
}
6272

6373
/**
64-
* The deserialization stream for SparkSqlSerializer2.
74+
* The corresponding deserialization stream for [[Serializer2SerializationStream]].
6575
*/
6676
private[sql] class Serializer2DeserializationStream(
6777
keySchema: Array[DataType],

0 commit comments

Comments
 (0)