diff options
author | Reynold Xin <rxin@apache.org> | 2013-10-20 12:14:59 -0700 |
---|---|---|
committer | Reynold Xin <rxin@apache.org> | 2013-10-20 12:22:07 -0700 |
commit | 8e1937f8ba999c875cc0d4403c4aa92d0a045de4 (patch) | |
tree | 84e27a30aa822790a99b78e03a41d07df5079eaf /core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala | |
parent | e4abb75d70aa08377829f635fe6135d94e28f434 (diff) | |
download | spark-8e1937f8ba999c875cc0d4403c4aa92d0a045de4.tar.gz spark-8e1937f8ba999c875cc0d4403c4aa92d0a045de4.tar.bz2 spark-8e1937f8ba999c875cc0d4403c4aa92d0a045de4.zip |
Made the following traits/interfaces/classes non-public:
SparkHadoopWriter
SparkHadoopMapRedUtil
SparkHadoopMapReduceUtil
SparkHadoopUtil
PythonAccumulatorParam
JobLogger
BlockManagerSlaveActor
Diffstat (limited to 'core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala')
-rw-r--r-- | core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala | 16 |
1 files changed, 8 insertions, 8 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index afa76a4a76..103a1c2051 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -17,14 +17,14 @@ package org.apache.hadoop.mapred -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path - +import java.io.IOException import java.text.SimpleDateFormat import java.text.NumberFormat -import java.io.IOException import java.util.Date +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path + import org.apache.spark.Logging import org.apache.spark.SerializableWritable @@ -36,6 +36,7 @@ import org.apache.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. */ +private[apache] class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil @@ -86,13 +87,11 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } getOutputCommitter().setupTask(getTaskContext()) - writer = getOutputFormat().getRecordWriter( - fs, conf.value, outputName, Reporter.NULL) + writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) } def write(key: AnyRef, value: AnyRef) { - if (writer!=null) { - //println (">>> Writing ("+key.toString+": " + key.getClass.toString + ", " + value.toString + ": " + value.getClass.toString + ")") + if (writer != null) { writer.write(key, value) } else { throw new IOException("Writer is null, open() has not been called") @@ -182,6 +181,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } +private[apache] object SparkHadoopWriter { def createJobID(time: Date, id: Int): JobID = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") |