aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJey Kottalam <jey@cs.berkeley.edu>2013-08-06 16:16:07 -0700
committerJey Kottalam <jey@cs.berkeley.edu>2013-08-15 16:50:37 -0700
commita06a9d5c5fd6584a57292a0115253e0a8a45d490 (patch)
tree41dee4d5e81090072ebe15ee8b0550d67bb98528
parent8f979edef5b80967b81323e13dcafd5aac92feb1 (diff)
downloadspark-a06a9d5c5fd6584a57292a0115253e0a8a45d490.tar.gz
spark-a06a9d5c5fd6584a57292a0115253e0a8a45d490.tar.bz2
spark-a06a9d5c5fd6584a57292a0115253e0a8a45d490.zip
Rename HadoopWriter to SparkHadoopWriter since it's outside of our package
-rw-r--r--core/src/main/scala/spark/PairRDDFunctions.scala6
-rw-r--r--core/src/main/scala/spark/SparkHadoopWriter.scala (renamed from core/src/main/scala/spark/HadoopWriter.scala)6
2 files changed, 6 insertions, 6 deletions
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index aeeac65cca..6701f24ff9 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.mapred.FileOutputCommitter
import org.apache.hadoop.mapred.FileOutputFormat
-import org.apache.hadoop.mapred.HadoopWriter
+import org.apache.hadoop.mapred.SparkHadoopWriter
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.OutputFormat
@@ -653,7 +653,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString)
}
conf.setOutputCommitter(classOf[FileOutputCommitter])
- FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf))
+ FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf))
saveAsHadoopDataset(conf)
}
@@ -679,7 +679,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")")
- val writer = new HadoopWriter(conf)
+ val writer = new SparkHadoopWriter(conf)
writer.preSetup()
def writeToFile(context: TaskContext, iter: Iterator[(K,V)]) {
diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/SparkHadoopWriter.scala
index 60840ce77e..6b330ef572 100644
--- a/core/src/main/scala/spark/HadoopWriter.scala
+++ b/core/src/main/scala/spark/SparkHadoopWriter.scala
@@ -36,7 +36,7 @@ import spark.SerializableWritable
* Saves the RDD using a JobConf, which should contain an output key class, an output value class,
* a filename to write to, etc, exactly like in a Hadoop MapReduce job.
*/
-class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable {
+class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable {
private val now = new Date()
private val conf = new SerializableWritable(jobConf)
@@ -165,7 +165,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoop
splitID = splitid
attemptID = attemptid
- jID = new SerializableWritable[JobID](HadoopWriter.createJobID(now, jobid))
+ jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid))
taID = new SerializableWritable[TaskAttemptID](
new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
}
@@ -179,7 +179,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoop
}
}
-object HadoopWriter {
+object SparkHadoopWriter {
def createJobID(time: Date, id: Int): JobID = {
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
val jobtrackerID = formatter.format(new Date())