diff options
author | Andrew Or <andrew@databricks.com> | 2014-10-30 17:56:10 -0700 |
---|---|---|
committer | Andrew Or <andrew@databricks.com> | 2014-10-30 17:56:10 -0700 |
commit | 26d31d15fda3f63707a28d1a1115770ad127cf8f (patch) | |
tree | d0249e55597c59f8a525caa5e707c12e4e99fd87 /core | |
parent | 2e35e24294ad8a5e76c89ea888fe330052dabd5a (diff) | |
download | spark-26d31d15fda3f63707a28d1a1115770ad127cf8f.tar.gz spark-26d31d15fda3f63707a28d1a1115770ad127cf8f.tar.bz2 spark-26d31d15fda3f63707a28d1a1115770ad127cf8f.zip |
Revert "SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop"
This reverts commit 68cb69daf3022e973422e496ccf827ca3806ff30.
Diffstat (limited to 'core')
-rw-r--r-- | core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala (renamed from core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala) | 6 | ||||
-rw-r--r-- | core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala (renamed from core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala) | 5 | ||||
-rw-r--r-- | core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 | ||||
-rw-r--r-- | core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 | ||||
-rw-r--r-- | core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 |
5 files changed, 5 insertions, 11 deletions
diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala index fe2bc65f89..0c47afae54 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.mapred +package org.apache.hadoop.mapred -import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} - -private[spark] +private[apache] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index 3340673f91..1fca5729c6 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.mapreduce +package org.apache.hadoop.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[spark] +private[apache] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 4023759657..376e69cd99 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index e7b1170755..3245632487 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,7 +35,6 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 462f0d6268..da89f634ab 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,14 +33,13 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter} +RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils |