From e4ad8b5230c64d9c0224231454d4fbc739c16279 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sat, 20 Dec 2014 21:26:36 +0800 Subject: [PATCH] Use null for the implicit Ordering --- .../main/scala/org/apache/spark/rdd/RDD.scala | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index bd17dae14d3a8..a94206963b52f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1175,18 +1175,19 @@ abstract class RDD[T: ClassTag]( */ def saveAsTextFile(path: String) { // https://issues.apache.org/jira/browse/SPARK-2075 - // NullWritable is a Comparable rather than Comparable[NullWritable] in Hadoop 1.+, - // so the compiler cannot find an implicit Ordering for it and will use the default `null`. - // It will generate different anonymous classes for `saveAsTextFile` in Hadoop 1.+ and - // Hadoop 2.+. Therefore, here we provide an Ordering for NullWritable so that the compiler - // will generate same bytecode. - val nullWritableOrdering = new Ordering[NullWritable] { - override def compare(x: NullWritable, y: NullWritable): Int = 0 - } + // + // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit + // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]` + // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an + // Ordering for `NullWritable`. That's why the compiler will generate different anonymous + // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. + // + // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate + // same bytecodes for `saveAsTextFile`. val nullWritableClassTag = implicitly[ClassTag[NullWritable]] val textClassTag = implicitly[ClassTag[Text]] val r = this.map(x => (NullWritable.get(), new Text(x.toString))) - RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, nullWritableOrdering) + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } @@ -1195,13 +1196,10 @@ abstract class RDD[T: ClassTag]( */ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { // https://issues.apache.org/jira/browse/SPARK-2075 - val nullWritableOrdering = new Ordering[NullWritable] { - override def compare(x: NullWritable, y: NullWritable): Int = 0 - } val nullWritableClassTag = implicitly[ClassTag[NullWritable]] val textClassTag = implicitly[ClassTag[Text]] val r = this.map(x => (NullWritable.get(), new Text(x.toString))) - RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, nullWritableOrdering) + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) }