-
- Downloads
[SPARK-2075][Core] Make the compiler generate same bytes code for Hadoop 1.+ and Hadoop 2.+
`NullWritable` is a `Comparable` rather than `Comparable[NullWritable]` in Hadoop 1.+, so the compiler cannot find an implicit Ordering for it. 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 codes. I used the following commands to confirm the generated byte codes are some. ``` mvn -Dhadoop.version=1.2.1 -DskipTests clean package -pl core -am javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop1.txt mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -pl core -am javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop2.txt diff ~/hadoop1.txt ~/hadoop2.txt ``` However, the compiler will generate different codes for the classes which call methods of `JobContext/TaskAttemptContext`. `JobContext/TaskAttemptContext` is a class in Hadoop 1.+, and calling its method will use `invokevirtual`, while it's an interface in Hadoop 2.+, and will use `invokeinterface`. To fix it, we can use reflection to call `JobContext/TaskAttemptContext.getConfiguration`. Author: zsxwing <zsxwing@gmail.com> Closes #3740 from zsxwing/SPARK-2075 and squashes the following commits: 39d9df2 [zsxwing] Fix the code style e4ad8b5 [zsxwing] Use null for the implicit Ordering 734bac9 [zsxwing] Explicitly set the implicit parameters ca03559 [zsxwing] Use reflection to access JobContext/TaskAttemptContext.getConfiguration fa40db0 [zsxwing] Add an Ordering for NullWritable to make the compiler generate same byte codes for RDD
Showing
- core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 12 additions, 0 deletions.../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
- core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala 2 additions, 1 deletion...org/apache/spark/input/FixedLengthBinaryInputFormat.scala
- core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala 2 additions, 1 deletion...rg/apache/spark/input/FixedLengthBinaryRecordReader.scala
- core/src/main/scala/org/apache/spark/input/PortableDataStream.scala 3 additions, 1 deletion...ain/scala/org/apache/spark/input/PortableDataStream.scala
- core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala 3 additions, 1 deletion...la/org/apache/spark/input/WholeTextFileRecordReader.scala
- core/src/main/scala/org/apache/spark/rdd/RDD.scala 19 additions, 2 deletionscore/src/main/scala/org/apache/spark/rdd/RDD.scala
Loading
Please register or sign in to comment