Skip to content
Snippets Groups Projects
Commit ce6ed2ab authored by surq's avatar surq Committed by Tathagata Das
Browse files

[SPARK-3954][Streaming] Optimization to FileInputDStream

about convert files to RDDS there are 3 loops with files sequence in spark source.
loops files sequence:
1.files.map(...)
2.files.zip(fileRDDs)
3.files-size.foreach
It's will very time consuming when lots of files.So I do the following correction:
3 loops with files sequence => only one loop

Author: surq <surq@asiainfo.com>

Closes #2811 from surq/SPARK-3954 and squashes the following commits:

321bbe8 [surq]  updated the code style.The style from [for...yield]to [files.map(file=>{})]
88a2c20 [surq] Merge branch 'master' of https://github.com/apache/spark into SPARK-3954
178066f [surq] modify code's style. [Exceeds 100 columns]
626ef97 [surq] remove redundant import(ArrayBuffer)
739341f [surq] promote the speed of convert files to RDDS
parent a1fc059b
No related branches found
No related tags found
No related merge requests found
......@@ -120,14 +120,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
/** Generate one RDD from an array of files */
private def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
val fileRDDs = files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
files.zip(fileRDDs).foreach { case (file, rdd) => {
val fileRDDs = files.map(file =>{
val rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file)
if (rdd.partitions.size == 0) {
logError("File " + file + " has no data in it. Spark Streaming can only ingest " +
"files that have been \"moved\" to the directory assigned to the file stream. " +
"Refer to the streaming programming guide for more details.")
}
}}
rdd
})
new UnionRDD(context.sparkContext, fileRDDs)
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment