-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-1133] Add whole text files reader in MLlib #252
Closed
Closed
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
28cb0fe
add whole text files reader
yinxusen a1f1e7e
add two extra spaces
yinxusen 6bdf2c2
test for small local file system block size
yinxusen d792cee
remove the typo character "+"
yinxusen cc97dca
move whole text file API to Spark core
yinxusen 01745ee
fix deletion error
yinxusen 0af3faf
add JavaAPI test
yinxusen 7191be6
refine comments
yinxusen File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -37,6 +37,7 @@ import org.apache.mesos.MesosNativeLibrary | |
|
||
import org.apache.spark.broadcast.Broadcast | ||
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} | ||
import org.apache.spark.input.WholeTextFileInputFormat | ||
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} | ||
import org.apache.spark.rdd._ | ||
import org.apache.spark.scheduler._ | ||
|
@@ -371,6 +372,39 @@ class SparkContext( | |
minSplits).map(pair => pair._2.toString) | ||
} | ||
|
||
/** | ||
* Read a directory of text files from HDFS, a local file system (available on all nodes), or any | ||
* Hadoop-supported file system URI. Each file is read as a single record and returned in a | ||
* key-value pair, where the key is the path of each file, the value is the content of each file. | ||
* | ||
* <p> For example, if you have the following files: | ||
* {{{ | ||
* hdfs://a-hdfs-path/part-00000 | ||
* hdfs://a-hdfs-path/part-00001 | ||
* ... | ||
* hdfs://a-hdfs-path/part-nnnnn | ||
* }}} | ||
* | ||
* Do `val rdd = sparkContext.wholeTextFile("hdfs://a-hdfs-path")`, | ||
* | ||
* <p> then `rdd` contains | ||
* {{{ | ||
* (a-hdfs-path/part-00000, its content) | ||
* (a-hdfs-path/part-00001, its content) | ||
* ... | ||
* (a-hdfs-path/part-nnnnn, its content) | ||
* }}} | ||
* | ||
* @note Small files are perferred, large file is also allowable, but may cause bad performance. | ||
*/ | ||
def wholeTextFiles(path: String): RDD[(String, String)] = { | ||
newAPIHadoopFile( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. does this need to be wrapped? It looks like you could pull the lines up and still be < 100 characters. |
||
path, | ||
classOf[WholeTextFileInputFormat], | ||
classOf[String], | ||
classOf[String]) | ||
} | ||
|
||
/** | ||
* Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other | ||
* necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
47 changes: 47 additions & 0 deletions
47
core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.input | ||
|
||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.mapreduce.InputSplit | ||
import org.apache.hadoop.mapreduce.JobContext | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat | ||
import org.apache.hadoop.mapreduce.RecordReader | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit | ||
|
||
/** | ||
* A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat CombineFileInputFormat]] for | ||
* reading whole text files. Each file is read as key-value pair, where the key is the file path and | ||
* the value is the entire content of file. | ||
*/ | ||
|
||
private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] { | ||
override protected def isSplitable(context: JobContext, file: Path): Boolean = false | ||
|
||
override def createRecordReader( | ||
split: InputSplit, | ||
context: TaskAttemptContext): RecordReader[String, String] = { | ||
|
||
new CombineFileRecordReader[String, String]( | ||
split.asInstanceOf[CombineFileSplit], | ||
context, | ||
classOf[WholeTextFileRecordReader]) | ||
} | ||
} |
72 changes: 72 additions & 0 deletions
72
core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.input | ||
|
||
import com.google.common.io.{ByteStreams, Closeables} | ||
|
||
import org.apache.hadoop.io.Text | ||
import org.apache.hadoop.mapreduce.InputSplit | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit | ||
import org.apache.hadoop.mapreduce.RecordReader | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
|
||
/** | ||
* A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file | ||
* out in a key-value pair, where the key is the file path and the value is the entire content of | ||
* the file. | ||
*/ | ||
private[spark] class WholeTextFileRecordReader( | ||
split: CombineFileSplit, | ||
context: TaskAttemptContext, | ||
index: Integer) | ||
extends RecordReader[String, String] { | ||
|
||
private val path = split.getPath(index) | ||
private val fs = path.getFileSystem(context.getConfiguration) | ||
|
||
// True means the current file has been processed, then skip it. | ||
private var processed = false | ||
|
||
private val key = path.toString | ||
private var value: String = null | ||
|
||
override def initialize(split: InputSplit, context: TaskAttemptContext) = {} | ||
|
||
override def close() = {} | ||
|
||
override def getProgress = if (processed) 1.0f else 0.0f | ||
|
||
override def getCurrentKey = key | ||
|
||
override def getCurrentValue = value | ||
|
||
override def nextKeyValue = { | ||
if (!processed) { | ||
val fileIn = fs.open(path) | ||
val innerBuffer = ByteStreams.toByteArray(fileIn) | ||
|
||
value = new Text(innerBuffer).toString | ||
Closeables.close(fileIn, false) | ||
|
||
processed = true | ||
true | ||
} else { | ||
false | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
105 changes: 105 additions & 0 deletions
105
core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,105 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.input | ||
|
||
import java.io.DataOutputStream | ||
import java.io.File | ||
import java.io.FileOutputStream | ||
|
||
import scala.collection.immutable.IndexedSeq | ||
|
||
import com.google.common.io.Files | ||
|
||
import org.scalatest.BeforeAndAfterAll | ||
import org.scalatest.FunSuite | ||
|
||
import org.apache.hadoop.io.Text | ||
|
||
import org.apache.spark.SparkContext | ||
|
||
/** | ||
* Tests the correctness of | ||
* [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary | ||
* directory is created as fake input. Temporal storage would be deleted in the end. | ||
*/ | ||
class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { | ||
private var sc: SparkContext = _ | ||
|
||
override def beforeAll() { | ||
sc = new SparkContext("local", "test") | ||
|
||
// Set the block size of local file system to test whether files are split right or not. | ||
sc.hadoopConfiguration.setLong("fs.local.block.size", 32) | ||
} | ||
|
||
override def afterAll() { | ||
sc.stop() | ||
} | ||
|
||
private def createNativeFile(inputDir: File, fileName: String, contents: Array[Byte]) = { | ||
val out = new DataOutputStream(new FileOutputStream(s"${inputDir.toString}/$fileName")) | ||
out.write(contents, 0, contents.length) | ||
out.close() | ||
} | ||
|
||
/** | ||
* This code will test the behaviors of WholeTextFileRecordReader based on local disk. There are | ||
* three aspects to check: | ||
* 1) Whether all files are read; | ||
* 2) Whether paths are read correctly; | ||
* 3) Does the contents be the same. | ||
*/ | ||
test("Correctness of WholeTextFileRecordReader.") { | ||
|
||
val dir = Files.createTempDir() | ||
println(s"Local disk address is ${dir.toString}.") | ||
|
||
WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => | ||
createNativeFile(dir, filename, contents) | ||
} | ||
|
||
val res = sc.wholeTextFiles(dir.toString).collect() | ||
|
||
assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, | ||
"Number of files read out does not fit with the actual value.") | ||
|
||
for ((filename, contents) <- res) { | ||
val shortName = filename.split('/').last | ||
assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), | ||
s"Missing file name $filename.") | ||
assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, | ||
s"file $filename contents can not match.") | ||
} | ||
|
||
dir.delete() | ||
} | ||
} | ||
|
||
/** | ||
* Files to be tested are defined here. | ||
*/ | ||
object WholeTextFileRecordReaderSuite { | ||
private val testWords: IndexedSeq[Byte] = "Spark is easy to use.\n".map(_.toByte) | ||
|
||
private val fileNames = Array("part-00000", "part-00001", "part-00002") | ||
private val fileLengths = Array(10, 100, 1000) | ||
|
||
private val files = fileLengths.zip(fileNames).map { case (upperBound, filename) => | ||
filename -> Stream.continually(testWords.toList.toStream).flatten.take(upperBound).toArray | ||
}.toMap | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it make sense to add a warning that says to only use this for small files? (maybe it's obvious? :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe a warning is better, even though it can handle both big and small files, but the big files will cause bad performance.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, please add a warning at the end of this, and make sure to also put it in the Java API doc.