diff options
author | CodingCat <zhunansjtu@gmail.com> | 2014-03-24 21:55:03 -0700 |
---|---|---|
committer | Aaron Davidson <aaron@databricks.com> | 2014-03-24 21:55:03 -0700 |
commit | 5140598df889f7227c9d6a7953031eeef524badd (patch) | |
tree | 129e43867802653f6e29d48ff1ee6c2396392929 /sql/hive | |
parent | dc126f2121d0cd1dc0caa50ae0c4cb9137d42562 (diff) | |
download | spark-5140598df889f7227c9d6a7953031eeef524badd.tar.gz spark-5140598df889f7227c9d6a7953031eeef524badd.tar.bz2 spark-5140598df889f7227c9d6a7953031eeef524badd.zip |
SPARK-1128: set hadoop task properties when constructing HadoopRDD
https://spark-project.atlassian.net/browse/SPARK-1128
The task properties are not set when constructing HadoopRDD in current implementation, this may limit the implementation based on
```
mapred.tip.id
mapred.task.id
mapred.task.is.map
mapred.task.partition
mapred.job.id
```
This patch also contains a small fix in createJobID (SparkHadoopWriter.scala), where the current implementation actually is not using time parameter
Author: CodingCat <zhunansjtu@gmail.com>
Author: Nan Zhu <CodingCat@users.noreply.github.com>
Closes #101 from CodingCat/SPARK-1128 and squashes the following commits:
ed0980f [CodingCat] make SparkHiveHadoopWriter belongs to spark package
5b1ad7d [CodingCat] move SparkHiveHadoopWriter to org.apache.spark package
258f92c [CodingCat] code cleanup
af88939 [CodingCat] update the comments and permission of SparkHadoopWriter
9bd1fe3 [CodingCat] move configuration for jobConf to HadoopRDD
b7bdfa5 [Nan Zhu] style fix
a3153a8 [Nan Zhu] style fix
c3258d2 [CodingCat] set hadoop task properties while using InputFormat
Diffstat (limited to 'sql/hive')
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala (renamed from sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala) | 8 | ||||
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala | 2 |
2 files changed, 4 insertions, 6 deletions
diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 0b38731919..d96c2f70e0 100644 --- a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark import java.io.IOException import java.text.NumberFormat @@ -25,16 +25,14 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.mapred._ import org.apache.hadoop.io.Writable -import org.apache.spark.Logging -import org.apache.spark.SerializableWritable - /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected[apache] +protected[spark] class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index 9aa9e173a8..78f69e7ff5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -35,7 +35,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ -import org.apache.spark.{TaskContext, SparkException} +import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ |