Skip to content

Commit e4ad8b5

Browse files
committed
Use null for the implicit Ordering
1 parent 734bac9 commit e4ad8b5

File tree

1 file changed

+11
-13
lines changed
  • core/src/main/scala/org/apache/spark/rdd

1 file changed

+11
-13
lines changed

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 11 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -1175,18 +1175,19 @@ abstract class RDD[T: ClassTag](
11751175
*/
11761176
def saveAsTextFile(path: String) {
11771177
// https://issues.apache.org/jira/browse/SPARK-2075
1178-
// NullWritable is a Comparable rather than Comparable[NullWritable] in Hadoop 1.+,
1179-
// so the compiler cannot find an implicit Ordering for it and will use the default `null`.
1180-
// It will generate different anonymous classes for `saveAsTextFile` in Hadoop 1.+ and
1181-
// Hadoop 2.+. Therefore, here we provide an Ordering for NullWritable so that the compiler
1182-
// will generate same bytecode.
1183-
val nullWritableOrdering = new Ordering[NullWritable] {
1184-
override def compare(x: NullWritable, y: NullWritable): Int = 0
1185-
}
1178+
//
1179+
// NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit
1180+
// Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]`
1181+
// in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an
1182+
// Ordering for `NullWritable`. That's why the compiler will generate different anonymous
1183+
// classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+.
1184+
//
1185+
// Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate
1186+
// same bytecodes for `saveAsTextFile`.
11861187
val nullWritableClassTag = implicitly[ClassTag[NullWritable]]
11871188
val textClassTag = implicitly[ClassTag[Text]]
11881189
val r = this.map(x => (NullWritable.get(), new Text(x.toString)))
1189-
RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, nullWritableOrdering)
1190+
RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null)
11901191
.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path)
11911192
}
11921193

@@ -1195,13 +1196,10 @@ abstract class RDD[T: ClassTag](
11951196
*/
11961197
def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) {
11971198
// https://issues.apache.org/jira/browse/SPARK-2075
1198-
val nullWritableOrdering = new Ordering[NullWritable] {
1199-
override def compare(x: NullWritable, y: NullWritable): Int = 0
1200-
}
12011199
val nullWritableClassTag = implicitly[ClassTag[NullWritable]]
12021200
val textClassTag = implicitly[ClassTag[Text]]
12031201
val r = this.map(x => (NullWritable.get(), new Text(x.toString)))
1204-
RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, nullWritableOrdering)
1202+
RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null)
12051203
.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec)
12061204
}
12071205

0 commit comments

Comments
 (0)