-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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-13149][SQL]Add FileStreamSource #11034
Changes from 2 commits
a2784ff
6a90c55
9f5967f
937b86f
2af6fc8
f38153b
1ccea0f
ca1e6b8
d911963
ce0556d
9fd21c2
9a1042c
91a2b74
1ffee5f
fb0e3f9
07e2ddd
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 |
---|---|---|
@@ -0,0 +1,141 @@ | ||
/* | ||
* 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.sql.execution.streaming | ||
|
||
import java.io.{BufferedWriter, OutputStreamWriter} | ||
|
||
import scala.collection.mutable.ArrayBuffer | ||
|
||
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} | ||
|
||
import org.apache.spark.Logging | ||
import org.apache.spark.sql.{DataFrame, SQLContext} | ||
import org.apache.spark.sql.types.{StringType, StructType} | ||
import org.apache.spark.util.collection.OpenHashSet | ||
|
||
/** | ||
* A very simple source that reads text files from the given directory as they appear. | ||
*/ | ||
class FileStreamSource( | ||
sqlContext: SQLContext, | ||
metadataPath: String, | ||
path: String, | ||
dataSchema: Option[StructType], | ||
dataFrameBuilder: Array[String] => DataFrame) extends Source with Logging { | ||
|
||
import sqlContext.implicits._ | ||
|
||
/** Returns the schema of the data from this source */ | ||
override def schema: StructType = dataSchema.getOrElse(new StructType().add("value", StringType)) | ||
|
||
/** Returns the maximum offset that can be retrieved from the source. */ | ||
def fetchMaxOffset(): LongOffset = synchronized { | ||
val filesPresent = fetchAllFiles() | ||
val newFiles = new ArrayBuffer[String]() | ||
filesPresent.foreach { file => | ||
if (!seenFiles.contains(file)) { | ||
logDebug(s"new file: $file") | ||
newFiles.append(file) | ||
seenFiles.add(file) | ||
} else { | ||
logDebug(s"old file: $file") | ||
} | ||
} | ||
|
||
if (newFiles.nonEmpty) { | ||
maxBatchFile += 1 | ||
writeBatch(maxBatchFile, newFiles) | ||
} | ||
|
||
new LongOffset(maxBatchFile) | ||
} | ||
|
||
def currentOffset: LongOffset = synchronized { | ||
new LongOffset(maxBatchFile) | ||
} | ||
|
||
/** | ||
* Returns the next batch of data that is available after `start`, if any is available. | ||
*/ | ||
override def getNextBatch(start: Option[Offset]): Option[Batch] = { | ||
val startId = start.map(_.asInstanceOf[LongOffset].offset).getOrElse(-1L) | ||
val end = fetchMaxOffset() | ||
val endId = end.offset | ||
|
||
val batchFiles = (startId + 1 to endId).filter(_ >= 0).map(i => s"$metadataPath/$i") | ||
if (batchFiles.nonEmpty) { | ||
logDebug(s"Producing files from batches ${startId + 1}:$endId") | ||
logDebug(s"Batch files: $batchFiles") | ||
|
||
// Probably does not need to be a spark job... | ||
val files = sqlContext | ||
.read | ||
.text(batchFiles: _*) | ||
.as[String] | ||
.collect() | ||
logDebug(s"Streaming ${files.mkString(", ")}") | ||
Some(new Batch(end, dataFrameBuilder(files))) | ||
} else { | ||
None | ||
} | ||
} | ||
|
||
private def sparkContext = sqlContext.sparkContext | ||
|
||
private val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
private val existingBatchFiles = fetchAllBatchFiles() | ||
private val existingBatchIds = existingBatchFiles.map(_.getPath.getName.toInt) | ||
private var maxBatchFile = if (existingBatchIds.isEmpty) -1 else existingBatchIds.max | ||
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. nit: Isnt this |
||
private val seenFiles = new OpenHashSet[String] | ||
|
||
if (existingBatchFiles.nonEmpty) { | ||
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 add some comments on what this Spark job is for? |
||
sqlContext.read | ||
.text(existingBatchFiles.map(_.getPath.toString): _*) | ||
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 these be two indented rather than four indented? |
||
.as[String] | ||
.collect() | ||
.foreach { file => | ||
seenFiles.add(file) | ||
} | ||
} | ||
|
||
private def fetchAllBatchFiles(): Seq[FileStatus] = { | ||
try fs.listStatus(new Path(metadataPath)) catch { | ||
case _: java.io.FileNotFoundException => | ||
fs.mkdirs(new Path(metadataPath)) | ||
Seq.empty | ||
} | ||
} | ||
|
||
private def fetchAllFiles(): Seq[String] = { | ||
fs.listStatus(new Path(path)) | ||
.filterNot(_.getPath.getName.startsWith("_")) | ||
.map(_.getPath.toUri.toString) | ||
} | ||
|
||
private def writeBatch(id: Int, files: Seq[String]): Unit = { | ||
val path = new Path(metadataPath + "/" + id) | ||
val fs = FileSystem.get(sparkContext.hadoopConfiguration) | ||
val writer = new BufferedWriter(new OutputStreamWriter(fs.create(path, true))) | ||
files.foreach { file => | ||
writer.write(file) | ||
writer.write("\n") | ||
} | ||
writer.close() | ||
logDebug(s"Wrote batch file $path") | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -33,8 +33,8 @@ object LastOptions { | |
class DefaultSource extends StreamSourceProvider with StreamSinkProvider { | ||
override def createSource( | ||
sqlContext: SQLContext, | ||
parameters: Map[String, String], | ||
schema: Option[StructType]): Source = { | ||
schema: Option[StructType], | ||
parameters: Map[String, String]): Source = { | ||
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 we have to add stuff we should probably add a schema too. |
||
LastOptions.parameters = parameters | ||
LastOptions.schema = schema | ||
new Source { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,131 @@ | ||
/* | ||
* 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.sql.streaming | ||
|
||
import java.io.File | ||
|
||
import org.apache.spark.sql.StreamTest | ||
import org.apache.spark.sql.catalyst.util._ | ||
import org.apache.spark.sql.execution.datasources.ResolvedDataSource | ||
import org.apache.spark.sql.execution.streaming.{FileStreamSource, Offset} | ||
import org.apache.spark.sql.test.SharedSQLContext | ||
import org.apache.spark.util.Utils | ||
|
||
class FileStreamSourceSuite extends StreamTest with SharedSQLContext { | ||
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. Could we write these tests with the 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 also need to add a test that completely recreates the |
||
|
||
import testImplicits._ | ||
|
||
case class AddTextFileData(source: FileStreamSource, content: String, src: File, tmp: File) | ||
extends AddData { | ||
|
||
override def addData(): Offset = { | ||
val file = Utils.tempFileWith(new File(tmp, "text")) | ||
stringToFile(file, content).renameTo(new File(src, file.getName)) | ||
source.currentOffset + 1 | ||
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 +1? 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. To wait until the next batch finishes. 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. I think there is a race here then. You've already moved the file into the directory right? 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. Ah, right. How about adding PauseStream and ResumeStream actions? So that we can make sure no batch will run when adding data here. |
||
} | ||
} | ||
|
||
case class AddParquetFileData( | ||
source: FileStreamSource, | ||
content: Seq[String], | ||
src: File, | ||
tmp: File) extends AddData { | ||
|
||
override def addData(): Offset = { | ||
val file = Utils.tempFileWith(new File(tmp, "parquet")) | ||
content.toDS().toDF().write.parquet(file.getCanonicalPath) | ||
file.renameTo(new File(src, file.getName)) | ||
source.currentOffset + 1 | ||
} | ||
} | ||
|
||
test("read from text files") { | ||
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 should add tests that start/stop the stream as well to test recovery. We should also consider creating a stress test. 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. Updated tests to add |
||
val src = Utils.createTempDir("streaming.src") | ||
val tmp = Utils.createTempDir("streaming.tmp") | ||
|
||
val textSource = ResolvedDataSource.createSource( | ||
sqlContext, | ||
userSpecifiedSchema = None, | ||
providerName = "text", | ||
options = Map("path" -> src.getCanonicalPath)).asInstanceOf[FileStreamSource] | ||
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. I'd prefer to use the 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. The problem is 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. I think that we need to have at least some tests that go through the user facing API. I would even consider just always extracting the source from the df.queryExecution.analyzed.collect { case s: Source => s }.head 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. Updated the tests to use DataFrameReader |
||
val df = textSource.toDF().filter($"value" contains "keep") | ||
val filtered = df | ||
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. while this extra line?? |
||
|
||
testStream(filtered)( | ||
AddTextFileData(textSource, "drop1\nkeep2\nkeep3", src, tmp), | ||
CheckAnswer("keep2", "keep3"), | ||
AddTextFileData(textSource, "drop4\nkeep5\nkeep6", src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6"), | ||
AddTextFileData(textSource, "drop7\nkeep8\nkeep9", src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") | ||
) | ||
|
||
Utils.deleteRecursively(src) | ||
Utils.deleteRecursively(tmp) | ||
} | ||
|
||
test("read from json files") { | ||
val src = Utils.createTempDir("streaming.src") | ||
val tmp = Utils.createTempDir("streaming.tmp") | ||
|
||
val textSource = ResolvedDataSource.createSource( | ||
sqlContext, | ||
userSpecifiedSchema = None, | ||
providerName = "json", | ||
options = Map("path" -> src.getCanonicalPath)).asInstanceOf[FileStreamSource] | ||
val df = textSource.toDF().filter($"value" contains "keep") | ||
val filtered = df | ||
|
||
testStream(filtered)( | ||
AddTextFileData(textSource, "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}", src, tmp), | ||
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. The column name here is 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. Fixed |
||
CheckAnswer("keep2", "keep3"), | ||
AddTextFileData(textSource, "{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6"), | ||
AddTextFileData(textSource, "{'c': 'drop7'}\n{'c': 'keep8'}\n{'c': 'keep9'}", src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") | ||
) | ||
|
||
Utils.deleteRecursively(src) | ||
Utils.deleteRecursively(tmp) | ||
} | ||
|
||
test("read from parquet files") { | ||
val src = Utils.createTempDir("streaming.src") | ||
val tmp = Utils.createTempDir("streaming.tmp") | ||
|
||
val fileSource = ResolvedDataSource.createSource( | ||
sqlContext, | ||
userSpecifiedSchema = None, | ||
providerName = "parquet", | ||
options = Map("path" -> src.getCanonicalPath)).asInstanceOf[FileStreamSource] | ||
val df = fileSource.toDF().filter($"value" contains "keep") | ||
val filtered = df | ||
|
||
testStream(filtered)( | ||
AddParquetFileData(fileSource, Seq("drop1", "keep2", "keep3"), src, tmp), | ||
CheckAnswer("keep2", "keep3"), | ||
AddParquetFileData(fileSource, Seq("drop4", "keep5", "keep6"), src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6"), | ||
AddParquetFileData(fileSource, Seq("drop7", "keep8", "keep9"), src, tmp), | ||
CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") | ||
) | ||
|
||
Utils.deleteRecursively(src) | ||
Utils.deleteRecursively(tmp) | ||
} | ||
} |
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
getOrElse
is only going to work for the text file data source, I think that for things like JSON we should probably try and initialize the source usingdataFrameBuilder
and extract the schema from there.We should also add tests that would catch a problem here.
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.
Updated the logic here. Now if there are any existing files, it will use them to infer the schema. And also added a test for it.