Skip to content

Commit

Permalink
promote the speed of convert files to RDDS
Browse files Browse the repository at this point in the history
  • Loading branch information
surongquan committed Oct 15, 2014
1 parent 293a0b5 commit 739341f
Showing 1 changed file with 4 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.UnionRDD
import org.apache.spark.streaming.{StreamingContext, Time}
import org.apache.spark.util.TimeStampedHashMap
import scala.collection.mutable.ArrayBuffer


private[streaming]
Expand Down Expand Up @@ -120,14 +121,14 @@ 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 = for (file <- files; rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file)) yield {
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)
}

Expand Down

0 comments on commit 739341f

Please sign in to comment.