-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-4969][STREAMING][PYTHON] Add binaryRecords to streaming #3803
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
8550c26
ecef0eb
fe4e803
36cb0fd
23dd69f
9398bcb
28bff9b
8b70fbc
2843e9d
94d90d0
1c739aa
029d49c
a4324a3
d3e75b2
becb344
fcb915c
317b6d1
3ceb684
7373f73
9a3715a
47560f4
b85bffc
14bca9a
34d20ef
c2cfa6d
30eba67
c4237b8
eba925c
5ff1b75
b676534
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 |
|---|---|---|
|
|
@@ -27,10 +27,12 @@ import scala.reflect.ClassTag | |
| import akka.actor.{Props, SupervisorStrategy} | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.{LongWritable, Text} | ||
| import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} | ||
| import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} | ||
| import org.apache.hadoop.mapreduce.lib.input.TextInputFormat | ||
| import org.apache.spark._ | ||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.input.FixedLengthBinaryInputFormat | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming.dstream._ | ||
|
|
@@ -359,6 +361,30 @@ class StreamingContext private[streaming] ( | |
| new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) | ||
| } | ||
|
|
||
| /** | ||
| * Create a input stream that monitors a Hadoop-compatible filesystem | ||
| * for new files and reads them using the given key-value types and input format. | ||
| * Files must be written to the monitored directory by "moving" them from another | ||
| * location within the same file system. File names starting with . are ignored. | ||
| * @param directory HDFS directory to monitor for new file | ||
| * @param filter Function to filter paths to process | ||
| * @param newFilesOnly Should process only new files and ignore existing files in the directory | ||
| * @param conf Hadoop configuration | ||
| * @tparam K Key type for reading HDFS file | ||
| * @tparam V Value type for reading HDFS file | ||
| * @tparam F Input format for reading HDFS file | ||
| */ | ||
| def fileStream[ | ||
|
Contributor
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. Can you make these parameters a superset of the parameters in the other fileStream? Otherwise it seems like people can either use a conf or use a filter
Contributor
Author
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. Fine with me, though would require wiring the extra arguments (
Contributor
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. Why the wiring to binaryRecords? Those two parameters are only relevant to
Contributor
Author
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. Sorry, meant exposing them to |
||
| K: ClassTag, | ||
| V: ClassTag, | ||
| F <: NewInputFormat[K, V]: ClassTag | ||
| ] (directory: String, | ||
| filter: Path => Boolean, | ||
| newFilesOnly: Boolean, | ||
| conf: Configuration): InputDStream[(K, V)] = { | ||
| new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly, Option(conf)) | ||
| } | ||
|
|
||
| /** | ||
| * Create a input stream that monitors a Hadoop-compatible filesystem | ||
| * for new files and reads them as text files (using key as LongWritable, value | ||
|
|
@@ -371,6 +397,37 @@ class StreamingContext private[streaming] ( | |
| fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * | ||
| * Create an input stream that monitors a Hadoop-compatible filesystem | ||
| * for new files and reads them as flat binary files, assuming a fixed length per record, | ||
| * generating one byte array per record. Files must be written to the monitored directory | ||
| * by "moving" them from another location within the same file system. File names | ||
| * starting with . are ignored. | ||
| * | ||
| * '''Note:''' We ensure that the byte array for each record in the | ||
| * resulting RDDs of the DStream has the provided record length. | ||
| * | ||
| * @param directory HDFS directory to monitor for new file | ||
| * @param recordLength length of each record in bytes | ||
| */ | ||
| @Experimental | ||
| def binaryRecordsStream( | ||
| directory: String, | ||
| recordLength: Int): DStream[Array[Byte]] = { | ||
| val conf = sc_.hadoopConfiguration | ||
| conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) | ||
| val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat]( | ||
| directory, FileInputDStream.defaultFilter : Path => Boolean, newFilesOnly=true, conf) | ||
| val data = br.map { case (k, v) => | ||
| val bytes = v.getBytes | ||
| assert(bytes.length == recordLength, "Byte array does not have correct length") | ||
| bytes | ||
| } | ||
| data | ||
| } | ||
|
|
||
| /** | ||
| * Create an input stream from a queue of RDDs. In each batch, | ||
| * it will process either one or all of the RDDs returned by the queue. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path | |
| import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} | ||
|
|
||
| import org.apache.spark.{SparkConf, SparkContext} | ||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} | ||
| import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} | ||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -177,7 +178,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { | |
|
|
||
| /** | ||
| * Create an input stream from network source hostname:port. Data is received using | ||
| * a TCP socket and the receive bytes it interepreted as object using the given | ||
| * a TCP socket and the receive bytes it interpreted as object using the given | ||
| * converter. | ||
| * @param hostname Hostname to connect to for receiving data | ||
| * @param port Port to connect to for receiving data | ||
|
|
@@ -209,6 +210,24 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { | |
| ssc.textFileStream(directory) | ||
| } | ||
|
|
||
| /** | ||
|
Contributor
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. Please expose the new version of filestream in javaStreamingContext. |
||
| * :: Experimental :: | ||
| * | ||
| * Create an input stream that monitors a Hadoop-compatible filesystem | ||
| * for new files and reads them as flat binary files with fixed record lengths, | ||
| * yielding byte arrays | ||
| * | ||
| * '''Note:''' We ensure that the byte array for each record in the | ||
| * resulting RDDs of the DStream has the provided record length. | ||
| * | ||
| * @param directory HDFS directory to monitor for new files | ||
| * @param recordLength The length at which to split the records | ||
| */ | ||
|
Contributor
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. Shouldnt this have the note as well?
Contributor
Author
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. Thanks, added! |
||
| @Experimental | ||
| def binaryRecordsStream(directory: String, recordLength: Int): JavaDStream[Array[Byte]] = { | ||
| ssc.binaryRecordsStream(directory, recordLength) | ||
| } | ||
|
|
||
| /** | ||
| * Create an input stream from network source hostname:port, where data is received | ||
| * as serialized blocks (serialized using the Spark's serializer) that can be directly | ||
|
|
@@ -298,6 +317,37 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { | |
| ssc.fileStream[K, V, F](directory, fn, newFilesOnly) | ||
| } | ||
|
|
||
| /** | ||
| * Create an input stream that monitors a Hadoop-compatible filesystem | ||
| * for new files and reads them using the given key-value types and input format. | ||
| * Files must be written to the monitored directory by "moving" them from another | ||
| * location within the same file system. File names starting with . are ignored. | ||
| * @param directory HDFS directory to monitor for new file | ||
| * @param kClass class of key for reading HDFS file | ||
| * @param vClass class of value for reading HDFS file | ||
| * @param fClass class of input format for reading HDFS file | ||
| * @param filter Function to filter paths to process | ||
| * @param newFilesOnly Should process only new files and ignore existing files in the directory | ||
| * @param conf Hadoop configuration | ||
| * @tparam K Key type for reading HDFS file | ||
| * @tparam V Value type for reading HDFS file | ||
| * @tparam F Input format for reading HDFS file | ||
| */ | ||
| def fileStream[K, V, F <: NewInputFormat[K, V]]( | ||
| directory: String, | ||
| kClass: Class[K], | ||
| vClass: Class[V], | ||
| fClass: Class[F], | ||
| filter: JFunction[Path, JBoolean], | ||
| newFilesOnly: Boolean, | ||
| conf: Configuration): JavaPairInputDStream[K, V] = { | ||
| implicit val cmk: ClassTag[K] = ClassTag(kClass) | ||
| implicit val cmv: ClassTag[V] = ClassTag(vClass) | ||
| implicit val cmf: ClassTag[F] = ClassTag(fClass) | ||
| def fn = (x: Path) => filter.call(x).booleanValue() | ||
| ssc.fileStream[K, V, F](directory, fn, newFilesOnly, conf) | ||
| } | ||
|
|
||
| /** | ||
| * Create an input stream with any arbitrary user implemented actor receiver. | ||
| * @param props Props object defining creation of the actor | ||
|
|
||
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.
nit: Is this something that the user should be made aware of in the docs?
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.
Do you mean something more than these notes we're adding? I just clarified the notes a bit to make it obvious the check is on the byte array.
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 meant should the user be told that the system can throw error when the records are not of the expected size. I dont have any strong feeling on this, just wondering.
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.
Gotcha, I think it's ok as is then. Given what
FixedLengthInputFormatis doing, this is more a defensive assertion, it's not something the user should hit due to an inappropriate input.