-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-1795 - Add recursive directory file search to fileInputStream #537
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.dstream | |
| import java.io.{ObjectInputStream, IOException} | ||
| import scala.collection.mutable.{HashSet, HashMap} | ||
| import scala.reflect.ClassTag | ||
| import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} | ||
| import org.apache.hadoop.fs.{FileSystem, Path, PathFilter, FileStatus} | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} | ||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -34,18 +34,19 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas | |
| @transient ssc_ : StreamingContext, | ||
| directory: String, | ||
| filter: Path => Boolean = FileInputDStream.defaultFilter, | ||
| newFilesOnly: Boolean = true) | ||
| newFilesOnly: Boolean = true, | ||
| recursive: Boolean = false) | ||
| extends InputDStream[(K, V)](ssc_) { | ||
|
|
||
| protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData | ||
|
|
||
| // files found in the last interval | ||
| private val lastFoundFiles = new HashSet[String] | ||
|
|
||
| // Files with mod time earlier than this is ignored. This is updated every interval | ||
| // such that in the current interval, files older than any file found in the | ||
| // previous interval will be ignored. Obviously this time keeps moving forward. | ||
| private var ignoreTime = if (newFilesOnly) 0L else System.currentTimeMillis() | ||
| private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L | ||
| private var recursiveMinTime = 0L | ||
|
|
||
| // Latest file mod time seen till any point of time | ||
| @transient private var path_ : Path = null | ||
|
|
@@ -96,6 +97,23 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas | |
| fileModTimes.clearOldValues(lastNewFileFindingTime - 1) | ||
| } | ||
|
|
||
| /** | ||
| * Find files recursively in a directory | ||
| */ | ||
| private def recursiveFileList( | ||
| fileStatuses: List[FileStatus], | ||
| paths: List[Path] = List[Path]() | ||
| ): List[Path] = fileStatuses match { | ||
|
|
||
| case f :: tail if (fs.getContentSummary(f.getPath).getDirectoryCount > 1) => | ||
| recursiveFileList(fs.listStatus(f.getPath).toList ::: tail, paths) | ||
| case f :: tail if f.isDir => recursiveFileList(tail, f.getPath :: paths) | ||
| case f :: tail => recursiveFileList(tail, paths) | ||
| case _ => paths | ||
|
|
||
| } | ||
|
|
||
|
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. If the given directory contains both files and directories then the files will be ignored and only lowest level of subdirectories will be considered. And if the data that to be considered is available in files in "/input/directory/" then it will be ignored. 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. how about this? |
||
|
|
||
| /** | ||
| * Find files which have modification timestamp <= current time and return a 3-tuple of | ||
| * (new files found, latest modification time among them, files with latest modification time) | ||
|
|
@@ -104,7 +122,14 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas | |
| logDebug("Trying to get new files for time " + currentTime) | ||
| lastNewFileFindingTime = System.currentTimeMillis | ||
| val filter = new CustomPathFilter(currentTime) | ||
| val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) | ||
|
|
||
| val filePaths: Array[Path] = if (recursive) | ||
| recursiveFileList(fs.listStatus(directoryPath).toList).toArray | ||
|
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. If the input directory is already the lowest level of directory then it will not consider any files in it. 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. We can call this like val filePaths: Array[Path] = if (recursive) |
||
| else | ||
| Array(directoryPath) | ||
|
|
||
| val newFiles: Array[String] = fs.listStatus(filePaths, filter).map(_.getPath.toString) | ||
|
|
||
| val timeTaken = System.currentTimeMillis - lastNewFileFindingTime | ||
| logInfo("Finding new files took " + timeTaken + " ms") | ||
| logDebug("# cached file times = " + fileModTimes.size) | ||
|
|
@@ -115,6 +140,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas | |
| "files in the monitored directory." | ||
| ) | ||
| } | ||
| logInfo("minNewFileModTime: " ++ filter.minNewFileModTime.toString) | ||
| (newFiles, filter.minNewFileModTime) | ||
| } | ||
|
|
||
|
|
@@ -218,6 +244,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas | |
| } | ||
| val modTime = getFileModTime(path) | ||
| logDebug("Mod time for " + path + " is " + modTime) | ||
|
|
||
| if (modTime < ignoreTime) { | ||
| // Reject file if it was created before the ignore time (or, before last interval) | ||
| logDebug("Mod time " + modTime + " less than ignore time " + ignoreTime) | ||
|
|
||
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.
This looks like an api change - please add default value to recursive
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.
I have included a default value on the FileInputDStream but not on the API itself.
Wondering if we want to introduce default values to the more granular version of the API. Currently, it looks like the exposed API essentially has two versions for these methods -- one that assumes default values and one that exposes all the parameters of the DStream constructor.
Thoughts?
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.
In which version of spark can we get the API with support for nested directory streaming?