Skip to content

Commit cc97dca

Browse files
committed
move whole text file API to Spark core
1 parent d792cee commit cc97dca

File tree

6 files changed

+64
-170
lines changed

6 files changed

+64
-170
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ import org.apache.mesos.MesosNativeLibrary
3737

3838
import org.apache.spark.broadcast.Broadcast
3939
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
40+
import org.apache.spark.input.WholeTextFileInputFormat
4041
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
4142
import org.apache.spark.rdd._
4243
import org.apache.spark.scheduler._
@@ -371,6 +372,37 @@ class SparkContext(
371372
minSplits).map(pair => pair._2.toString)
372373
}
373374

375+
/**
376+
* Read a directory of text files from HDFS, a local file system (available on all nodes), or any
377+
* Hadoop-supported file system URI. Each file is read as a single record and returned in a
378+
* key-value pair, where the key is the path of each file, the value is the content of each file.
379+
*
380+
* <p> For example, if you have the following files:
381+
* {{{
382+
* hdfs://a-hdfs-path/part-00000
383+
* hdfs://a-hdfs-path/part-00001
384+
* ...
385+
* hdfs://a-hdfs-path/part-nnnnn
386+
* }}}
387+
*
388+
* Do `val rdd = mlContext.wholeTextFile("hdfs://a-hdfs-path")`,
389+
*
390+
* <p> then `rdd` contains
391+
* {{{
392+
* (a-hdfs-path/part-00000, its content)
393+
* (a-hdfs-path/part-00001, its content)
394+
* ...
395+
* (a-hdfs-path/part-nnnnn, its content)
396+
* }}}
397+
*/
398+
def wholeTextFiles(path: String): RDD[(String, String)] = {
399+
newAPIHadoopFile(
400+
path,
401+
classOf[WholeTextFileInputFormat],
402+
classOf[String],
403+
classOf[String])
404+
}
405+
374406
/**
375407
* Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other
376408
* necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable),

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,31 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
154154
*/
155155
def textFile(path: String, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits)
156156

157+
/**
158+
* Read a directory of text files from HDFS, a local file system (available on all nodes), or any
159+
* Hadoop-supported file system URI. Each file is read as a single record and returned in a
160+
* key-value pair, where the key is the path of each file, the value is the content of each file.
161+
*
162+
* <p> For example, if you have the following files:
163+
* {{{
164+
* hdfs://a-hdfs-path/part-00000
165+
* hdfs://a-hdfs-path/part-00001
166+
* ...
167+
* hdfs://a-hdfs-path/part-nnnnn
168+
* }}}
169+
*
170+
* Do `val rdd = mlContext.wholeTextFile("hdfs://a-hdfs-path")`,
171+
*
172+
* <p> then `rdd` contains
173+
* {{{
174+
* (a-hdfs-path/part-00000, its content)
175+
* (a-hdfs-path/part-00001, its content)
176+
* ...
177+
* (a-hdfs-path/part-nnnnn, its content)
178+
* }}}
179+
*/
180+
def wholeTextFiles(path: String): JavaRDD[(String, String)] = sc.wholeTextFiles(path)
181+
157182
/** Get an RDD for a Hadoop SequenceFile with given key and value types.
158183
*
159184
* '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each

mllib/src/main/scala/org/apache/spark/mllib/input/WholeTextFileInputFormat.scala renamed to core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.mllib.input
18+
package org.apache.spark.input
1919

2020
import org.apache.hadoop.fs.Path
2121
import org.apache.hadoop.mapreduce.InputSplit
@@ -32,7 +32,7 @@ import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit
3232
* the value is the entire content of file.
3333
*/
3434

35-
private[mllib] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] {
35+
private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] {
3636
override protected def isSplitable(context: JobContext, file: Path): Boolean = false
3737

3838
override def createRecordReader(

mllib/src/main/scala/org/apache/spark/mllib/input/WholeTextFileRecordReader.scala renamed to core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.mllib.input
18+
package org.apache.spark.input
1919

2020
import com.google.common.io.{ByteStreams, Closeables}
2121

@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext
3030
* out in a key-value pair, where the key is the file path and the value is the entire content of
3131
* the file.
3232
*/
33-
private[mllib] class WholeTextFileRecordReader(
33+
private[spark] class WholeTextFileRecordReader(
3434
split: CombineFileSplit,
3535
context: TaskAttemptContext,
3636
index: Integer)

mllib/src/test/scala/org/apache/spark/mllib/input/WholeTextFileRecordReaderSuite.scala renamed to core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.mllib.input
18+
package org.apache.spark.input
1919

2020
import java.io.DataOutputStream
2121
import java.io.File
@@ -31,11 +31,10 @@ import org.scalatest.FunSuite
3131
import org.apache.hadoop.io.Text
3232

3333
import org.apache.spark.SparkContext
34-
import org.apache.spark.mllib.MLContext._
3534

3635
/**
3736
* Tests the correctness of
38-
* [[org.apache.spark.mllib.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary
37+
* [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary
3938
* directory is created as fake input. Temporal storage would be deleted in the end.
4039
*/
4140
class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll {
@@ -74,7 +73,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll {
7473
createNativeFile(dir, filename, contents)
7574
}
7675

77-
val res = sc.wholeTextFile(dir.toString).collect()
76+
val res = sc.wholeTextFiles(dir.toString).collect()
7877

7978
assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size,
8079
"Number of files read out does not fit with the actual value.")

mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala

Lines changed: 0 additions & 162 deletions
This file was deleted.

0 commit comments

Comments
 (0)